From 47b44c730efcd09337bc6b15152ce4c2cf88be99 Mon Sep 17 00:00:00 2001 From: Prasanth Jayachandran Date: Sat, 23 Jun 2018 17:27:00 -0700 Subject: [PATCH] HIVE-19926 : Remove deprecated hcatalog streaming --- hcatalog/pom.xml | 1 - hcatalog/streaming/pom.xml | 152 -- .../streaming/AbstractRecordWriter.java | 334 --- .../hcatalog/streaming/ConnectionError.java | 35 - .../streaming/DelimitedInputWriter.java | 333 --- .../hcatalog/streaming/HeartBeatFailure.java | 33 - .../hive/hcatalog/streaming/HiveEndPoint.java | 1060 -------- .../streaming/ImpersonationFailed.java | 25 - .../hcatalog/streaming/InvalidColumn.java | 26 - .../hcatalog/streaming/InvalidPartition.java | 28 - .../hive/hcatalog/streaming/InvalidTable.java | 38 - .../streaming/InvalidTrasactionState.java | 26 - .../streaming/PartitionCreationFailed.java | 25 - .../streaming/QueryFailedException.java | 28 - .../hive/hcatalog/streaming/RecordWriter.java | 47 - .../streaming/SerializationError.java | 26 - .../streaming/StreamingConnection.java | 59 - .../streaming/StreamingException.java | 28 - .../streaming/StreamingIOFailure.java | 31 - .../hcatalog/streaming/StrictJsonWriter.java | 164 -- .../hcatalog/streaming/StrictRegexWriter.java | 190 -- .../hcatalog/streaming/TransactionBatch.java | 126 - .../TransactionBatchUnAvailable.java | 25 - .../hcatalog/streaming/TransactionError.java | 29 - .../streaming/mutate/HiveConfFactory.java | 83 - .../mutate/UgiMetaStoreClientFactory.java | 121 - .../streaming/mutate/client/AcidTable.java | 131 - .../mutate/client/AcidTableSerializer.java | 119 - .../mutate/client/ClientException.java | 32 - .../mutate/client/ConnectionException.java | 32 - .../mutate/client/MutatorClient.java | 183 -- .../mutate/client/MutatorClientBuilder.java | 135 - .../streaming/mutate/client/TableType.java | 54 - .../streaming/mutate/client/Transaction.java | 135 - .../mutate/client/TransactionException.java | 32 - .../mutate/client/lock/HeartbeatFactory.java | 47 - .../client/lock/HeartbeatTimerTask.java | 83 - .../streaming/mutate/client/lock/Lock.java | 327 --- .../mutate/client/lock/LockException.java | 32 - .../client/lock/LockFailureListener.java | 46 - .../mutate/doc-files/system-overview.dot | 44 - .../hcatalog/streaming/mutate/package.html | 520 ---- .../mutate/worker/BucketIdException.java | 28 - .../mutate/worker/BucketIdResolver.java | 31 - .../mutate/worker/BucketIdResolverImpl.java | 93 - .../worker/GroupRevisitedException.java | 28 - .../mutate/worker/GroupingValidator.java | 91 - .../worker/MetaStorePartitionHelper.java | 119 - .../streaming/mutate/worker/Mutator.java | 40 - .../mutate/worker/MutatorCoordinator.java | 300 --- .../worker/MutatorCoordinatorBuilder.java | 121 - .../mutate/worker/MutatorFactory.java | 38 - .../streaming/mutate/worker/MutatorImpl.java | 114 - .../mutate/worker/OperationType.java | 24 - .../worker/PartitionCreationException.java | 32 - .../mutate/worker/PartitionHelper.java | 37 - .../mutate/worker/RecordInspector.java | 31 - .../mutate/worker/RecordInspectorImpl.java | 64 - .../worker/RecordSequenceException.java | 28 - .../mutate/worker/SequenceValidator.java | 66 - .../worker/WarehousePartitionHelper.java | 86 - .../mutate/worker/WorkerException.java | 32 - .../hive/hcatalog/streaming/package.html | 181 -- .../streaming/StreamingIntegrationTester.java | 347 --- .../streaming/TestDelimitedInputWriter.java | 71 - .../hcatalog/streaming/TestStreaming.java | 2373 ----------------- .../streaming/mutate/ExampleUseCase.java | 99 - .../streaming/mutate/MutableRecord.java | 50 - .../mutate/ReflectiveMutatorFactory.java | 68 - .../streaming/mutate/StreamingAssert.java | 223 -- .../streaming/mutate/StreamingTestUtils.java | 288 -- .../streaming/mutate/TestMutations.java | 566 ---- .../client/TestAcidTableSerializer.java | 83 - .../mutate/client/TestMutatorClient.java | 197 -- .../mutate/client/TestTransaction.java | 112 - .../client/lock/TestHeartbeatTimerTask.java | 117 - .../mutate/client/lock/TestLock.java | 339 --- .../worker/TestBucketIdResolverImpl.java | 59 - .../mutate/worker/TestGroupingValidator.java | 87 - .../worker/TestMetaStorePartitionHelper.java | 129 - .../mutate/worker/TestMutatorCoordinator.java | 262 -- .../mutate/worker/TestMutatorImpl.java | 116 - .../worker/TestRecordInspectorImpl.java | 48 - .../mutate/worker/TestSequenceValidator.java | 108 - .../worker/TestWarehousePartitionHelper.java | 74 - hcatalog/streaming/src/test/sit | 39 - itests/hive-unit/pom.xml | 4 - .../ql/txn/compactor/CompactorTestUtil.java | 21 - .../hive/ql/txn/compactor/TestCompactor.java | 223 +- packaging/pom.xml | 5 - packaging/src/main/assembly/bin.xml | 1 - 91 files changed, 52 insertions(+), 12666 deletions(-) delete mode 100644 hcatalog/streaming/pom.xml delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ConnectionError.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HeartBeatFailure.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ImpersonationFailed.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidColumn.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidPartition.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTable.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTrasactionState.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/PartitionCreationFailed.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/QueryFailedException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/SerializationError.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingIOFailure.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatchUnAvailable.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionError.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java delete mode 100644 hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package.html delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/StreamingIntegrationTester.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestDelimitedInputWriter.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java delete mode 100644 hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java delete mode 100644 hcatalog/streaming/src/test/sit diff --git a/hcatalog/pom.xml b/hcatalog/pom.xml index c1506d8dc2..fd01cbec9c 100644 --- a/hcatalog/pom.xml +++ b/hcatalog/pom.xml @@ -43,7 +43,6 @@ server-extensions webhcat/java-client webhcat/svr - streaming diff --git a/hcatalog/streaming/pom.xml b/hcatalog/streaming/pom.xml deleted file mode 100644 index fef5bf75ab..0000000000 --- a/hcatalog/streaming/pom.xml +++ /dev/null @@ -1,152 +0,0 @@ - - - - - 4.0.0 - - org.apache.hive.hcatalog - hive-hcatalog - 4.0.0-SNAPSHOT - ../pom.xml - - - hive-hcatalog-streaming - jar - Hive HCatalog Streaming - - - ../.. - - - - - - - org.apache.hive - hive-serde - ${project.version} - - - org.apache.hive - hive-metastore - ${project.version} - - - org.apache.hive - hive-exec - ${project.version} - - - org.apache.hive - hive-cli - ${project.version} - - - org.apache.hive.hcatalog - hive-hcatalog-core - true - ${project.version} - - - org.apache.commons - commons-lang3 - true - ${commons-lang3.version} - - - org.apache.hadoop - hadoop-common - true - ${hadoop.version} - - - commons-beanutils - commons-beanutils - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - true - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - - - - - junit - junit - test - - - - org.apache.hadoop - hadoop-mapreduce-client-common - test - - - org.slf4j - slf4j-log4j12 - - - commons-logging - commons-logging - - - - - - - - ${basedir}/src/java - ${basedir}/src/test - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - - diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java deleted file mode 100644 index bc99b6c824..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java +++ /dev/null @@ -1,334 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.security.UserGroupInformation; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.RecordUpdater; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hive.hcatalog.common.HCatUtil; -import org.apache.thrift.TException; - -import java.io.IOException; - -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Properties; - -/** - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.AbstractRecordWriter - */ -@Deprecated -public abstract class AbstractRecordWriter implements RecordWriter { - static final private Logger LOG = LoggerFactory.getLogger(AbstractRecordWriter.class.getName()); - - private final HiveConf conf; - private final HiveEndPoint endPoint; - final Table tbl; - - private final IMetaStoreClient msClient; - final List bucketIds; - private ArrayList updaters = null; - - private final int totalBuckets; - /** - * Indicates whether target table is bucketed - */ - private final boolean isBucketed; - - private final Path partitionPath; - - private final AcidOutputFormat outf; - private Object[] bucketFieldData; // Pre-allocated in constructor. Updated on each write. - private Long curBatchMinWriteId; - private Long curBatchMaxWriteId; - - private static final class TableWriterPair { - private final Table tbl; - private final Path partitionPath; - TableWriterPair(Table t, Path p) { - tbl = t; - partitionPath = p; - } - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #AbstractRecordWriter(HiveEndPoint, HiveConf, StreamingConnection)} - */ - protected AbstractRecordWriter(HiveEndPoint endPoint, HiveConf conf) - throws ConnectionError, StreamingException { - this(endPoint, conf, null); - } - protected AbstractRecordWriter(HiveEndPoint endPoint2, HiveConf conf, StreamingConnection conn) - throws StreamingException { - this.endPoint = endPoint2; - this.conf = conf!=null ? conf - : HiveEndPoint.createHiveConf(DelimitedInputWriter.class, endPoint.metaStoreUri); - try { - msClient = HCatUtil.getHiveMetastoreClient(this.conf); - UserGroupInformation ugi = conn != null ? conn.getUserGroupInformation() : null; - if (ugi == null) { - this.tbl = msClient.getTable(endPoint.database, endPoint.table); - this.partitionPath = getPathForEndPoint(msClient, endPoint); - } else { - TableWriterPair twp = ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public TableWriterPair run() throws Exception { - return new TableWriterPair(msClient.getTable(endPoint.database, endPoint.table), - getPathForEndPoint(msClient, endPoint)); - } - }); - this.tbl = twp.tbl; - this.partitionPath = twp.partitionPath; - } - this.isBucketed = tbl.getSd().getNumBuckets() > 0; - /** - * For unbucketed tables we have exactly 1 RecrodUpdater for each AbstractRecordWriter which - * ends up writing to a file bucket_000000 - * See also {@link #getBucket(Object)} - */ - this.totalBuckets = isBucketed ? tbl.getSd().getNumBuckets() : 1; - if(isBucketed) { - this.bucketIds = getBucketColIDs(tbl.getSd().getBucketCols(), tbl.getSd().getCols()); - this.bucketFieldData = new Object[bucketIds.size()]; - } - else { - bucketIds = Collections.emptyList(); - } - String outFormatName = this.tbl.getSd().getOutputFormat(); - outf = (AcidOutputFormat) ReflectionUtils.newInstance(JavaUtils.loadClass(outFormatName), conf); - } catch(InterruptedException e) { - throw new StreamingException(endPoint2.toString(), e); - } catch (MetaException | NoSuchObjectException e) { - throw new ConnectionError(endPoint2, e); - } catch (TException | ClassNotFoundException | IOException e) { - throw new StreamingException(e.getMessage(), e); - } - } - - /** - * used to tag error msgs to provied some breadcrumbs - */ - String getWatermark() { - return partitionPath + " writeIds[" + curBatchMinWriteId + "," + curBatchMaxWriteId + "]"; - } - // return the column numbers of the bucketed columns - private List getBucketColIDs(List bucketCols, List cols) { - ArrayList result = new ArrayList(bucketCols.size()); - HashSet bucketSet = new HashSet(bucketCols); - for (int i = 0; i < cols.size(); i++) { - if( bucketSet.contains(cols.get(i).getName()) ) { - result.add(i); - } - } - return result; - } - - /** - * Get the SerDe for the Objects created by {@link #encode}. This is public so that test - * frameworks can use it. - * @return serde - * @throws SerializationError - */ - public abstract AbstractSerDe getSerde() throws SerializationError; - - /** - * Encode a record as an Object that Hive can read with the ObjectInspector associated with the - * serde returned by {@link #getSerde}. This is public so that test frameworks can use it. - * @param record record to be deserialized - * @return deserialized record as an Object - * @throws SerializationError - */ - public abstract Object encode(byte[] record) throws SerializationError; - - protected abstract ObjectInspector[] getBucketObjectInspectors(); - protected abstract StructObjectInspector getRecordObjectInspector(); - protected abstract StructField[] getBucketStructFields(); - - // returns the bucket number to which the record belongs to - protected int getBucket(Object row) throws SerializationError { - if(!isBucketed) { - return 0; - } - ObjectInspector[] inspectors = getBucketObjectInspectors(); - Object[] bucketFields = getBucketFields(row); - int bucketingVersion = Utilities.getBucketingVersion( - tbl.getParameters().get(hive_metastoreConstants.TABLE_BUCKETING_VERSION)); - - return bucketingVersion == 2 ? - ObjectInspectorUtils.getBucketNumber(bucketFields, inspectors, totalBuckets) : - ObjectInspectorUtils.getBucketNumberOld(bucketFields, inspectors, totalBuckets); - } - - @Override - public void flush() throws StreamingIOFailure { - try { - for (RecordUpdater updater : updaters) { - if (updater != null) { - updater.flush(); - } - } - } catch (IOException e) { - throw new StreamingIOFailure("Unable to flush recordUpdater", e); - } - } - - @Override - public void clear() throws StreamingIOFailure { - } - - /** - * Creates a new record updater for the new batch - * @param minWriteId smallest writeid in the batch - * @param maxWriteID largest writeid in the batch - * @throws StreamingIOFailure if failed to create record updater - */ - @Override - public void newBatch(Long minWriteId, Long maxWriteID) - throws StreamingIOFailure, SerializationError { - curBatchMinWriteId = minWriteId; - curBatchMaxWriteId = maxWriteID; - updaters = new ArrayList(totalBuckets); - for (int bucket = 0; bucket < totalBuckets; bucket++) { - updaters.add(bucket, null);//so that get(i) returns null rather than ArrayOutOfBounds - } - } - - @Override - public void closeBatch() throws StreamingIOFailure { - boolean haveError = false; - for (RecordUpdater updater : updaters) { - if (updater != null) { - try { - //try not to leave any files open - updater.close(false); - } catch (Exception ex) { - haveError = true; - LOG.error("Unable to close " + updater + " due to: " + ex.getMessage(), ex); - } - } - } - updaters.clear(); - if(haveError) { - throw new StreamingIOFailure("Encountered errors while closing (see logs) " + getWatermark()); - } - } - - protected static ObjectInspector[] getObjectInspectorsForBucketedCols(List bucketIds - , StructObjectInspector recordObjInspector) - throws SerializationError { - ObjectInspector[] result = new ObjectInspector[bucketIds.size()]; - - for (int i = 0; i < bucketIds.size(); i++) { - int bucketId = bucketIds.get(i); - result[i] = - recordObjInspector.getAllStructFieldRefs().get( bucketId ).getFieldObjectInspector(); - } - return result; - } - - - private Object[] getBucketFields(Object row) throws SerializationError { - StructObjectInspector recordObjInspector = getRecordObjectInspector(); - StructField[] bucketStructFields = getBucketStructFields(); - for (int i = 0; i < bucketIds.size(); i++) { - bucketFieldData[i] = recordObjInspector.getStructFieldData(row, bucketStructFields[i]); - } - return bucketFieldData; - } - - private RecordUpdater createRecordUpdater(int bucketId, Long minWriteId, Long maxWriteID) - throws IOException, SerializationError { - try { - // Initialize table properties from the table parameters. This is required because the table - // may define certain table parameters that may be required while writing. The table parameter - // 'transactional_properties' is one such example. - Properties tblProperties = new Properties(); - tblProperties.putAll(tbl.getParameters()); - return outf.getRecordUpdater(partitionPath, - new AcidOutputFormat.Options(conf) - .inspector(getSerde().getObjectInspector()) - .bucket(bucketId) - .tableProperties(tblProperties) - .minimumWriteId(minWriteId) - .maximumWriteId(maxWriteID) - .statementId(-1) - .finalDestination(partitionPath)); - } catch (SerDeException e) { - throw new SerializationError("Failed to get object inspector from Serde " - + getSerde().getClass().getName(), e); - } - } - - RecordUpdater getRecordUpdater(int bucketId) throws StreamingIOFailure, SerializationError { - RecordUpdater recordUpdater = updaters.get(bucketId); - if (recordUpdater == null) { - try { - recordUpdater = createRecordUpdater(bucketId, curBatchMinWriteId, curBatchMaxWriteId); - } catch (IOException e) { - String errMsg = "Failed creating RecordUpdater for " + getWatermark(); - LOG.error(errMsg, e); - throw new StreamingIOFailure(errMsg, e); - } - updaters.set(bucketId, recordUpdater); - } - return recordUpdater; - } - - private Path getPathForEndPoint(IMetaStoreClient msClient, HiveEndPoint endPoint) - throws StreamingException { - try { - String location; - if(endPoint.partitionVals==null || endPoint.partitionVals.isEmpty() ) { - location = msClient.getTable(endPoint.database,endPoint.table) - .getSd().getLocation(); - } else { - location = msClient.getPartition(endPoint.database, endPoint.table, - endPoint.partitionVals).getSd().getLocation(); - } - return new Path(location); - } catch (TException e) { - throw new StreamingException(e.getMessage() - + ". Unable to get path for end point: " - + endPoint.partitionVals, e); - } - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ConnectionError.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ConnectionError.java deleted file mode 100644 index 897b82634b..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ConnectionError.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class ConnectionError extends StreamingException { - - public ConnectionError(String msg) { - super(msg); - } - - public ConnectionError(String msg, Exception innerEx) { - super(msg, innerEx); - } - - public ConnectionError(HiveEndPoint endPoint, Exception innerEx) { - super("Error connecting to " + endPoint + - (innerEx == null ? "" : ": " + innerEx.getMessage()), innerEx); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java deleted file mode 100644 index 85c3429329..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java +++ /dev/null @@ -1,333 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -import com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters; -import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; -import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.io.BytesWritable; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -/** - * Streaming Writer handles delimited input (eg. CSV). - * Delimited input is parsed & reordered to match column order in table - * Uses Lazy Simple Serde to process delimited input - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.StrictDelimitedInputWriter - */ -@Deprecated -public class DelimitedInputWriter extends AbstractRecordWriter { - private final boolean reorderingNeeded; - private String delimiter; - private char serdeSeparator; - private int[] fieldToColMapping; - private final ArrayList tableColumns; - private LazySimpleSerDe serde = null; - - private final LazySimpleStructObjectInspector recordObjInspector; - private final ObjectInspector[] bucketObjInspectors; - private final StructField[] bucketStructFields; - - static final private Logger LOG = LoggerFactory.getLogger(DelimitedInputWriter.class.getName()); - - /** Constructor. Uses default separator of the LazySimpleSerde - * @param colNamesForFields Column name assignment for input fields. nulls or empty - * strings in the array indicates the fields to be skipped - * @param delimiter input field delimiter - * @param endPoint Hive endpoint - * @throws ConnectionError Problem talking to Hive - * @throws ClassNotFoundException Serde class not found - * @throws SerializationError Serde initialization/interaction failed - * @throws StreamingException Problem acquiring file system path for partition - * @throws InvalidColumn any element in colNamesForFields refers to a non existing column - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint, StreamingConnection conn) - throws ClassNotFoundException, ConnectionError, SerializationError, - InvalidColumn, StreamingException { - this(colNamesForFields, delimiter, endPoint, null, conn); - } - /** Constructor. Uses default separator of the LazySimpleSerde - * @param colNamesForFields Column name assignment for input fields. nulls or empty - * strings in the array indicates the fields to be skipped - * @param delimiter input field delimiter - * @param endPoint Hive endpoint - * @param conf a Hive conf object. Can be null if not using advanced hive settings. - * @throws ConnectionError Problem talking to Hive - * @throws ClassNotFoundException Serde class not found - * @throws SerializationError Serde initialization/interaction failed - * @throws StreamingException Problem acquiring file system path for partition - * @throws InvalidColumn any element in colNamesForFields refers to a non existing column - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint, HiveConf conf, StreamingConnection conn) - throws ClassNotFoundException, ConnectionError, SerializationError, - InvalidColumn, StreamingException { - this(colNamesForFields, delimiter, endPoint, conf, - (char) LazySerDeParameters.DefaultSeparators[0], conn); - } - /** - * Constructor. Allows overriding separator of the LazySimpleSerde - * @param colNamesForFields Column name assignment for input fields - * @param delimiter input field delimiter - * @param endPoint Hive endpoint - * @param conf a Hive conf object. Set to null if not using advanced hive settings. - * @param serdeSeparator separator used when encoding data that is fed into the - * LazySimpleSerde. Ensure this separator does not occur - * in the field data - * @param conn connection this Writer is to be used with - * @throws ConnectionError Problem talking to Hive - * @throws ClassNotFoundException Serde class not found - * @throws SerializationError Serde initialization/interaction failed - * @throws StreamingException Problem acquiring file system path for partition - * @throws InvalidColumn any element in colNamesForFields refers to a non existing column - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint, HiveConf conf, char serdeSeparator, StreamingConnection conn) - throws ClassNotFoundException, ConnectionError, SerializationError, - InvalidColumn, StreamingException { - super(endPoint, conf, conn); - this.tableColumns = getCols(tbl); - this.serdeSeparator = serdeSeparator; - this.delimiter = delimiter; - this.fieldToColMapping = getFieldReordering(colNamesForFields, getTableColumns()); - this.reorderingNeeded = isReorderingNeeded(delimiter, getTableColumns()); - LOG.debug("Field reordering needed = " + this.reorderingNeeded + ", for endpoint " + endPoint); - this.serdeSeparator = serdeSeparator; - this.serde = createSerde(tbl, conf, serdeSeparator); - - // get ObjInspectors for entire record and bucketed cols - try { - this.recordObjInspector = (LazySimpleStructObjectInspector) serde.getObjectInspector(); - this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, recordObjInspector); - } catch (SerDeException e) { - throw new SerializationError("Unable to get ObjectInspector for bucket columns", e); - } - - // get StructFields for bucketed cols - bucketStructFields = new StructField[bucketIds.size()]; - List allFields = recordObjInspector.getAllStructFieldRefs(); - for (int i = 0; i < bucketIds.size(); i++) { - bucketStructFields[i] = allFields.get(bucketIds.get(i)); - } - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #DelimitedInputWriter(String[], String, HiveEndPoint, StreamingConnection)} - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint) - throws ClassNotFoundException, ConnectionError, SerializationError, - InvalidColumn, StreamingException { - this(colNamesForFields, delimiter, endPoint, null, null); - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #DelimitedInputWriter(String[], String, HiveEndPoint, HiveConf, StreamingConnection)} - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint, HiveConf conf) - throws ClassNotFoundException, ConnectionError, SerializationError, - InvalidColumn, StreamingException { - this(colNamesForFields, delimiter, endPoint, conf, - (char) LazySerDeParameters.DefaultSeparators[0], null); - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #DelimitedInputWriter(String[], String, HiveEndPoint, HiveConf, char, StreamingConnection)} - */ - public DelimitedInputWriter(String[] colNamesForFields, String delimiter, - HiveEndPoint endPoint, HiveConf conf, char serdeSeparator) - throws ClassNotFoundException, StreamingException { - this(colNamesForFields, delimiter, endPoint, conf, serdeSeparator, null); - } - - private boolean isReorderingNeeded(String delimiter, ArrayList tableColumns) { - return !( delimiter.equals(String.valueOf(getSerdeSeparator())) - && areFieldsInColOrder(fieldToColMapping) - && tableColumns.size()>=fieldToColMapping.length ); - } - - private static boolean areFieldsInColOrder(int[] fieldToColMapping) { - for(int i=0; i tableColNames) - throws InvalidColumn { - int[] result = new int[ colNamesForFields.length ]; - for(int i=0; itableColNames.size()) { - throw new InvalidColumn("Number of field names exceeds the number of columns in table"); - } - return result; - } - - // Reorder fields in record based on the order of columns in the table - protected byte[] reorderFields(byte[] record) throws UnsupportedEncodingException { - if(!reorderingNeeded) { - return record; - } - String[] reorderedFields = new String[getTableColumns().size()]; - String decoded = new String(record); - String[] fields = decoded.split(delimiter,-1); - for (int i=0; i getTableColumns() { - return tableColumns; - } - - @Override - public void write(long writeId, byte[] record) - throws SerializationError, StreamingIOFailure { - try { - byte[] orderedFields = reorderFields(record); - Object encodedRow = encode(orderedFields); - int bucket = getBucket(encodedRow); - getRecordUpdater(bucket).insert(writeId, encodedRow); - } catch (IOException e) { - throw new StreamingIOFailure("Error writing record in transaction write id (" - + writeId + ")", e); - } - } - - @Override - public AbstractSerDe getSerde() { - return serde; - } - - protected LazySimpleStructObjectInspector getRecordObjectInspector() { - return recordObjInspector; - } - - @Override - protected StructField[] getBucketStructFields() { - return bucketStructFields; - } - - protected ObjectInspector[] getBucketObjectInspectors() { - return bucketObjInspectors; - } - - @Override - public Object encode(byte[] record) throws SerializationError { - try { - BytesWritable blob = new BytesWritable(); - blob.set(record, 0, record.length); - return serde.deserialize(blob); - } catch (SerDeException e) { - throw new SerializationError("Unable to convert byte[] record into Object", e); - } - } - - /** - * Creates LazySimpleSerde - * @return - * @throws SerializationError if serde could not be initialized - * @param tbl - */ - protected static LazySimpleSerDe createSerde(Table tbl, HiveConf conf, char serdeSeparator) - throws SerializationError { - try { - Properties tableProps = MetaStoreUtils.getTableMetadata(tbl); - tableProps.setProperty("field.delim", String.valueOf(serdeSeparator)); - LazySimpleSerDe serde = new LazySimpleSerDe(); - SerDeUtils.initializeSerDe(serde, conf, tableProps, null); - return serde; - } catch (SerDeException e) { - throw new SerializationError("Error initializing serde", e); - } - } - - private ArrayList getCols(Table table) { - List cols = table.getSd().getCols(); - ArrayList colNames = new ArrayList(cols.size()); - for (FieldSchema col : cols) { - colNames.add(col.getName().toLowerCase()); - } - return colNames; - } - - public char getSerdeSeparator() { - return serdeSeparator; - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HeartBeatFailure.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HeartBeatFailure.java deleted file mode 100644 index 5d9b763c84..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HeartBeatFailure.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import java.util.Collection; -import java.util.Set; - -public class HeartBeatFailure extends StreamingException { - private Collection abortedTxns; - private Collection nosuchTxns; - - public HeartBeatFailure(Collection abortedTxns, Set nosuchTxns) { - super("Heart beat error. InvalidTxns: " + nosuchTxns + ". AbortedTxns: " + abortedTxns); - this.abortedTxns = abortedTxns; - this.nosuchTxns = nosuchTxns; - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java deleted file mode 100644 index dc8c6636bc..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java +++ /dev/null @@ -1,1060 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.DataOperationType; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.LockComponentBuilder; -import org.apache.hadoop.hive.metastore.LockRequestBuilder; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockState; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.hadoop.hive.metastore.api.TxnToWriteId; -import org.apache.hive.hcatalog.common.HCatUtil; - -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.thrift.TException; - -import java.io.IOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -/** - * Information about the hive end point (i.e. table or partition) to write to. - * A light weight object that does NOT internally hold on to resources such as - * network connections. It can be stored in Hashed containers such as sets and hash tables. - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.HiveStreamingConnection - */ -@Deprecated -public class HiveEndPoint { - public final String metaStoreUri; - public final String database; - public final String table; - public final ArrayList partitionVals; - - - static final private Logger LOG = LoggerFactory.getLogger(HiveEndPoint.class.getName()); - - /** - * - * @param metaStoreUri URI of the metastore to connect to eg: thrift://localhost:9083 - * @param database Name of the Hive database - * @param table Name of table to stream to - * @param partitionVals Indicates the specific partition to stream to. Can be null or empty List - * if streaming to a table without partitions. The order of values in this - * list must correspond exactly to the order of partition columns specified - * during the table creation. E.g. For a table partitioned by - * (continent string, country string), partitionVals could be the list - * ("Asia", "India"). - */ - public HiveEndPoint(String metaStoreUri - , String database, String table, List partitionVals) { - this.metaStoreUri = metaStoreUri; - if (database==null) { - throw new IllegalArgumentException("Database cannot be null for HiveEndPoint"); - } - this.database = database.toLowerCase(); - if (table==null) { - throw new IllegalArgumentException("Table cannot be null for HiveEndPoint"); - } - this.partitionVals = partitionVals==null ? new ArrayList() - : new ArrayList( partitionVals ); - this.table = table.toLowerCase(); - } - - - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #newConnection(boolean, String)} - */ - @Deprecated - public StreamingConnection newConnection(final boolean createPartIfNotExists) - throws ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed - , ImpersonationFailed , InterruptedException { - return newConnection(createPartIfNotExists, null, null, null); - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #newConnection(boolean, HiveConf, String)} - */ - @Deprecated - public StreamingConnection newConnection(final boolean createPartIfNotExists, HiveConf conf) - throws ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed - , ImpersonationFailed , InterruptedException { - return newConnection(createPartIfNotExists, conf, null, null); - } - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #newConnection(boolean, HiveConf, UserGroupInformation, String)} - */ - @Deprecated - public StreamingConnection newConnection(final boolean createPartIfNotExists, final HiveConf conf, - final UserGroupInformation authenticatedUser) - throws ConnectionError, InvalidPartition, - InvalidTable, PartitionCreationFailed, ImpersonationFailed , InterruptedException { - return newConnection(createPartIfNotExists, conf, authenticatedUser, null); - } - /** - * Acquire a new connection to MetaStore for streaming - * @param createPartIfNotExists If true, the partition specified in the endpoint - * will be auto created if it does not exist - * @param agentInfo should uniquely identify the process/entity that is using this batch. This - * should be something that can be correlated with calling application log files - * and/or monitoring consoles. - * @return - * @throws ConnectionError if problem connecting - * @throws InvalidPartition if specified partition is not valid (createPartIfNotExists = false) - * @throws ImpersonationFailed if not able to impersonate 'proxyUser' - * @throws PartitionCreationFailed if failed to create partition - * @throws InterruptedException - */ - public StreamingConnection newConnection(final boolean createPartIfNotExists, String agentInfo) - throws ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed - , ImpersonationFailed , InterruptedException { - return newConnection(createPartIfNotExists, null, null, agentInfo); - } - - /** - * Acquire a new connection to MetaStore for streaming - * @param createPartIfNotExists If true, the partition specified in the endpoint - * will be auto created if it does not exist - * @param conf HiveConf object, set it to null if not using advanced hive settings. - * @param agentInfo should uniquely identify the process/entity that is using this batch. This - * should be something that can be correlated with calling application log files - * and/or monitoring consoles. - * @return - * @throws ConnectionError if problem connecting - * @throws InvalidPartition if specified partition is not valid (createPartIfNotExists = false) - * @throws ImpersonationFailed if not able to impersonate 'proxyUser' - * @throws PartitionCreationFailed if failed to create partition - * @throws InterruptedException - */ - public StreamingConnection newConnection(final boolean createPartIfNotExists, HiveConf conf, String agentInfo) - throws ConnectionError, InvalidPartition, InvalidTable, PartitionCreationFailed - , ImpersonationFailed , InterruptedException { - return newConnection(createPartIfNotExists, conf, null, agentInfo); - } - - /** - * Acquire a new connection to MetaStore for streaming. To connect using Kerberos, - * 'authenticatedUser' argument should have been used to do a kerberos login. Additionally the - * 'hive.metastore.kerberos.principal' setting should be set correctly either in hive-site.xml or - * in the 'conf' argument (if not null). If using hive-site.xml, it should be in classpath. - * - * @param createPartIfNotExists If true, the partition specified in the endpoint - * will be auto created if it does not exist - * @param conf HiveConf object to be used for the connection. Can be null. - * @param authenticatedUser UserGroupInformation object obtained from successful authentication. - * Uses non-secure mode if this argument is null. - * @param agentInfo should uniquely identify the process/entity that is using this batch. This - * should be something that can be correlated with calling application log files - * and/or monitoring consoles. - * @return - * @throws ConnectionError if there is a connection problem - * @throws InvalidPartition if specified partition is not valid (createPartIfNotExists = false) - * @throws ImpersonationFailed if not able to impersonate 'username' - * @throws PartitionCreationFailed if failed to create partition - * @throws InterruptedException - */ - public StreamingConnection newConnection(final boolean createPartIfNotExists, final HiveConf conf, - final UserGroupInformation authenticatedUser, final String agentInfo) - throws ConnectionError, InvalidPartition, - InvalidTable, PartitionCreationFailed, ImpersonationFailed , InterruptedException { - - if( authenticatedUser==null ) { - return newConnectionImpl(authenticatedUser, createPartIfNotExists, conf, agentInfo); - } - - try { - return authenticatedUser.doAs ( - new PrivilegedExceptionAction() { - @Override - public StreamingConnection run() - throws ConnectionError, InvalidPartition, InvalidTable - , PartitionCreationFailed { - return newConnectionImpl(authenticatedUser, createPartIfNotExists, conf, agentInfo); - } - } - ); - } catch (IOException e) { - throw new ConnectionError("Failed to connect as : " + authenticatedUser.getShortUserName(), e); - } - } - - private StreamingConnection newConnectionImpl(UserGroupInformation ugi, - boolean createPartIfNotExists, HiveConf conf, String agentInfo) - throws ConnectionError, InvalidPartition, InvalidTable - , PartitionCreationFailed { - return new ConnectionImpl(this, ugi, conf, createPartIfNotExists, agentInfo); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - HiveEndPoint endPoint = (HiveEndPoint) o; - - if (database != null - ? !database.equals(endPoint.database) - : endPoint.database != null ) { - return false; - } - if (metaStoreUri != null - ? !metaStoreUri.equals(endPoint.metaStoreUri) - : endPoint.metaStoreUri != null ) { - return false; - } - if (!partitionVals.equals(endPoint.partitionVals)) { - return false; - } - if (table != null ? !table.equals(endPoint.table) : endPoint.table != null) { - return false; - } - - return true; - } - - @Override - public int hashCode() { - int result = metaStoreUri != null ? metaStoreUri.hashCode() : 0; - result = 31 * result + (database != null ? database.hashCode() : 0); - result = 31 * result + (table != null ? table.hashCode() : 0); - result = 31 * result + partitionVals.hashCode(); - return result; - } - - @Override - public String toString() { - return "{" + - "metaStoreUri='" + metaStoreUri + '\'' + - ", database='" + database + '\'' + - ", table='" + table + '\'' + - ", partitionVals=" + partitionVals + " }"; - } - - - private static class ConnectionImpl implements StreamingConnection { - private final IMetaStoreClient msClient; - private final IMetaStoreClient heartbeaterMSClient; - private final HiveEndPoint endPt; - private final UserGroupInformation ugi; - private final String username; - private final boolean secureMode; - private final String agentInfo; - - /** - * @param endPoint end point to connect to - * @param ugi on behalf of whom streaming is done. cannot be null - * @param conf HiveConf object - * @param createPart create the partition if it does not exist - * @throws ConnectionError if there is trouble connecting - * @throws InvalidPartition if specified partition does not exist (and createPart=false) - * @throws InvalidTable if specified table does not exist - * @throws PartitionCreationFailed if createPart=true and not able to create partition - */ - private ConnectionImpl(HiveEndPoint endPoint, UserGroupInformation ugi, - HiveConf conf, boolean createPart, String agentInfo) - throws ConnectionError, InvalidPartition, InvalidTable - , PartitionCreationFailed { - this.endPt = endPoint; - this.ugi = ugi; - this.agentInfo = agentInfo; - this.username = ugi == null ? System.getProperty("user.name") : ugi.getShortUserName(); - if (conf == null) { - conf = HiveEndPoint.createHiveConf(this.getClass(), endPoint.metaStoreUri); - } else { - overrideConfSettings(conf); - } - this.secureMode = ugi == null ? false : ugi.hasKerberosCredentials(); - this.msClient = getMetaStoreClient(endPoint, conf, secureMode); - // We use a separate metastore client for heartbeat calls to ensure heartbeat RPC calls are - // isolated from the other transaction related RPC calls. - this.heartbeaterMSClient = getMetaStoreClient(endPoint, conf, secureMode); - checkEndPoint(endPoint, msClient); - if (createPart && !endPoint.partitionVals.isEmpty()) { - createPartitionIfNotExists(endPoint, msClient, conf); - } - } - - /** - * Checks the validity of endpoint - * - * @param endPoint the HiveEndPoint to be checked - * @param msClient the metastore client - * @throws InvalidTable - */ - private void checkEndPoint(HiveEndPoint endPoint, IMetaStoreClient msClient) - throws InvalidTable, ConnectionError { - Table t; - try { - t = msClient.getTable(endPoint.database, endPoint.table); - } catch (Exception e) { - LOG.warn("Unable to check the endPoint: " + endPoint, e); - throw new InvalidTable(endPoint.database, endPoint.table, e); - } - // 1 - check that the table is Acid - if (!AcidUtils.isFullAcidTable(t)) { - LOG.error("HiveEndPoint " + endPoint + " must use an acid table"); - throw new InvalidTable(endPoint.database, endPoint.table, "is not an Acid table"); - } - - // 2 - check if partitionvals are legitimate - if (t.getPartitionKeys() != null && !t.getPartitionKeys().isEmpty() - && endPoint.partitionVals.isEmpty()) { - // Invalid if table is partitioned, but endPoint's partitionVals is empty - String errMsg = "HiveEndPoint " + endPoint + " doesn't specify any partitions for " + - "partitioned table"; - LOG.error(errMsg); - throw new ConnectionError(errMsg); - } - if ((t.getPartitionKeys() == null || t.getPartitionKeys().isEmpty()) - && !endPoint.partitionVals.isEmpty()) { - // Invalid if table is not partitioned, but endPoint's partitionVals is not empty - String errMsg = "HiveEndPoint" + endPoint + " specifies partitions for unpartitioned table"; - LOG.error(errMsg); - throw new ConnectionError(errMsg); - } - } - - /** - * Close connection - */ - @Override - public void close() { - if (ugi == null) { - msClient.close(); - heartbeaterMSClient.close(); - return; - } - try { - ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public Void run() throws Exception { - msClient.close(); - heartbeaterMSClient.close(); - return null; - } - }); - try { - FileSystem.closeAllForUGI(ugi); - } catch (IOException exception) { - LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception); - } - } catch (IOException e) { - LOG.error("Error closing connection to " + endPt, e); - } catch (InterruptedException e) { - LOG.error("Interrupted when closing connection to " + endPt, e); - } - } - - @Override - public UserGroupInformation getUserGroupInformation() { - return ugi; - } - - /** - * Acquires a new batch of transactions from Hive. - * - * @param numTransactions is a hint from client indicating how many transactions client needs. - * @param recordWriter Used to write record. The same writer instance can - * be shared with another TransactionBatch (to the same endpoint) - * only after the first TransactionBatch has been closed. - * Writer will be closed when the TransactionBatch is closed. - * @return - * @throws StreamingIOFailure if failed to create new RecordUpdater for batch - * @throws TransactionBatchUnAvailable if failed to acquire a new Transaction batch - * @throws ImpersonationFailed failed to run command as proxyUser - * @throws InterruptedException - */ - @Override - public TransactionBatch fetchTransactionBatch(final int numTransactions, - final RecordWriter recordWriter) - throws StreamingException, TransactionBatchUnAvailable, ImpersonationFailed - , InterruptedException { - if (ugi == null) { - return fetchTransactionBatchImpl(numTransactions, recordWriter); - } - try { - return ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public TransactionBatch run() throws StreamingException, InterruptedException { - return fetchTransactionBatchImpl(numTransactions, recordWriter); - } - } - ); - } catch (IOException e) { - throw new ImpersonationFailed("Failed to fetch Txn Batch as user '" + ugi.getShortUserName() - + "' when acquiring Transaction Batch on endPoint " + endPt, e); - } - } - - private TransactionBatch fetchTransactionBatchImpl(int numTransactions, - RecordWriter recordWriter) - throws StreamingException, TransactionBatchUnAvailable, InterruptedException { - return new TransactionBatchImpl(username, ugi, endPt, numTransactions, msClient, - heartbeaterMSClient, recordWriter, agentInfo); - } - - private static void createPartitionIfNotExists(HiveEndPoint ep, - IMetaStoreClient msClient, HiveConf conf) throws PartitionCreationFailed { - if (ep.partitionVals.isEmpty()) { - return; - } - - try { - org.apache.hadoop.hive.ql.metadata.Table tableObject = - new org.apache.hadoop.hive.ql.metadata.Table(msClient.getTable(ep.database, ep.table)); - Map partSpec = - Warehouse.makeSpecFromValues(tableObject.getPartitionKeys(), ep.partitionVals); - - Path location = new Path(tableObject.getDataLocation(), Warehouse.makePartPath(partSpec)); - location = new Path(Utilities.getQualifiedPath(conf, location)); - Partition partition = - org.apache.hadoop.hive.ql.metadata.Partition.createMetaPartitionObject(tableObject, partSpec, location); - msClient.add_partition(partition); - } - catch (AlreadyExistsException e) { - //ignore this - multiple clients may be trying to create the same partition - //AddPartitionDesc has ifExists flag but it's not propagated to - // HMSHnalder.add_partitions_core() and so it throws... - } - catch(HiveException|TException e) { - LOG.error("Failed to create partition : " + ep, e); - throw new PartitionCreationFailed(ep, e); - } - } - - private static IMetaStoreClient getMetaStoreClient(HiveEndPoint endPoint, HiveConf conf, boolean secureMode) - throws ConnectionError { - - if (endPoint.metaStoreUri!= null) { - conf.setVar(HiveConf.ConfVars.METASTOREURIS, endPoint.metaStoreUri); - } - if(secureMode) { - conf.setBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL,true); - } - try { - return HCatUtil.getHiveMetastoreClient(conf); - } catch (MetaException e) { - throw new ConnectionError("Error connecting to Hive Metastore URI: " - + endPoint.metaStoreUri + ". " + e.getMessage(), e); - } catch (IOException e) { - throw new ConnectionError("Error connecting to Hive Metastore URI: " - + endPoint.metaStoreUri + ". " + e.getMessage(), e); - } - } - } // class ConnectionImpl - - private static class TransactionBatchImpl implements TransactionBatch { - private final String username; - private final UserGroupInformation ugi; - private final HiveEndPoint endPt; - private final IMetaStoreClient msClient; - private final IMetaStoreClient heartbeaterMSClient; - private final RecordWriter recordWriter; - private final List txnToWriteIds; - - //volatile because heartbeat() may be in a "different" thread; updates of this are "piggybacking" - private volatile int currentTxnIndex = -1; - private final String partNameForLock; - //volatile because heartbeat() may be in a "different" thread - private volatile TxnState state; - private LockRequest lockRequest = null; - /** - * once any operation on this batch encounters a system exception - * (e.g. IOException on write) it's safest to assume that we can't write to the - * file backing this batch any more. This guards important public methods - */ - private volatile boolean isClosed = false; - private final String agentInfo; - /** - * Tracks the state of each transaction - */ - private final TxnState[] txnStatus; - /** - * ID of the last txn used by {@link #beginNextTransactionImpl()} - */ - private long lastTxnUsed; - - /** - * Represents a batch of transactions acquired from MetaStore - * - * @throws StreamingException if failed to create new RecordUpdater for batch - * @throws TransactionBatchUnAvailable if failed to acquire a new Transaction batch - */ - private TransactionBatchImpl(final String user, UserGroupInformation ugi, HiveEndPoint endPt, - final int numTxns, final IMetaStoreClient msClient, - final IMetaStoreClient heartbeaterMSClient, RecordWriter recordWriter, String agentInfo) - throws StreamingException, TransactionBatchUnAvailable, InterruptedException { - boolean success = false; - try { - if ( endPt.partitionVals!=null && !endPt.partitionVals.isEmpty() ) { - Table tableObj = msClient.getTable(endPt.database, endPt.table); - List partKeys = tableObj.getPartitionKeys(); - partNameForLock = Warehouse.makePartName(partKeys, endPt.partitionVals); - } else { - partNameForLock = null; - } - this.username = user; - this.ugi = ugi; - this.endPt = endPt; - this.msClient = msClient; - this.heartbeaterMSClient = heartbeaterMSClient; - this.recordWriter = recordWriter; - this.agentInfo = agentInfo; - - List txnIds = openTxnImpl(msClient, user, numTxns, ugi); - txnToWriteIds = allocateWriteIdsImpl(msClient, txnIds, ugi); - assert(txnToWriteIds.size() == numTxns); - - txnStatus = new TxnState[numTxns]; - for(int i = 0; i < txnStatus.length; i++) { - assert(txnToWriteIds.get(i).getTxnId() == txnIds.get(i)); - txnStatus[i] = TxnState.OPEN;//Open matches Metastore state - } - this.state = TxnState.INACTIVE; - - // The Write Ids returned for the transaction batch is also sequential - recordWriter.newBatch(txnToWriteIds.get(0).getWriteId(), txnToWriteIds.get(numTxns-1).getWriteId()); - success = true; - } catch (TException e) { - throw new TransactionBatchUnAvailable(endPt, e); - } catch (IOException e) { - throw new TransactionBatchUnAvailable(endPt, e); - } - finally { - //clean up if above throws - markDead(success); - } - } - - private List openTxnImpl(final IMetaStoreClient msClient, final String user, final int numTxns, UserGroupInformation ugi) - throws IOException, TException, InterruptedException { - if(ugi==null) { - return msClient.openTxns(user, numTxns).getTxn_ids(); - } - return (List) ugi.doAs(new PrivilegedExceptionAction() { - @Override - public Object run() throws Exception { - return msClient.openTxns(user, numTxns).getTxn_ids(); - } - }); - } - - private List allocateWriteIdsImpl(final IMetaStoreClient msClient, - final List txnIds, UserGroupInformation ugi) - throws IOException, TException, InterruptedException { - if(ugi==null) { - return msClient.allocateTableWriteIdsBatch(txnIds, endPt.database, endPt.table); - } - return (List) ugi.doAs(new PrivilegedExceptionAction() { - @Override - public Object run() throws Exception { - return msClient.allocateTableWriteIdsBatch(txnIds, endPt.database, endPt.table); - } - }); - } - - @Override - public String toString() { - if (txnToWriteIds==null || txnToWriteIds.isEmpty()) { - return "{}"; - } - StringBuilder sb = new StringBuilder(" TxnStatus["); - for(TxnState state : txnStatus) { - //'state' should not be null - future proofing - sb.append(state == null ? "N" : state); - } - sb.append("] LastUsed ").append(JavaUtils.txnIdToString(lastTxnUsed)); - return "TxnId/WriteIds=[" + txnToWriteIds.get(0).getTxnId() - + "/" + txnToWriteIds.get(0).getWriteId() - + "..." - + txnToWriteIds.get(txnToWriteIds.size()-1).getTxnId() - + "/" + txnToWriteIds.get(txnToWriteIds.size()-1).getWriteId() - + "] on endPoint = " + endPt + "; " + sb; - } - - /** - * Activate the next available transaction in the current transaction batch - * @throws TransactionError failed to switch to next transaction - */ - @Override - public void beginNextTransaction() throws TransactionError, ImpersonationFailed, - InterruptedException { - checkIsClosed(); - if (ugi==null) { - beginNextTransactionImpl(); - return; - } - try { - ugi.doAs ( - new PrivilegedExceptionAction() { - @Override - public Void run() throws TransactionError { - beginNextTransactionImpl(); - return null; - } - } - ); - } catch (IOException e) { - throw new ImpersonationFailed("Failed switching to next Txn as user '" + username + - "' in Txn batch :" + this, e); - } - } - - private void beginNextTransactionImpl() throws TransactionError { - state = TxnState.INACTIVE;//clear state from previous txn - - if ((currentTxnIndex + 1) >= txnToWriteIds.size()) { - throw new InvalidTrasactionState("No more transactions available in" + - " current batch for end point : " + endPt); - } - ++currentTxnIndex; - state = TxnState.OPEN; - lastTxnUsed = getCurrentTxnId(); - lockRequest = createLockRequest(endPt, partNameForLock, username, getCurrentTxnId(), agentInfo); - try { - LockResponse res = msClient.lock(lockRequest); - if (res.getState() != LockState.ACQUIRED) { - throw new TransactionError("Unable to acquire lock on " + endPt); - } - } catch (TException e) { - throw new TransactionError("Unable to acquire lock on " + endPt, e); - } - } - - /** - * Get Id of currently open transaction. - * @return -1 if there is no open TX - */ - @Override - public Long getCurrentTxnId() { - if (currentTxnIndex >= 0) { - return txnToWriteIds.get(currentTxnIndex).getTxnId(); - } - return -1L; - } - - /** - * Get Id of currently open transaction. - * @return -1 if there is no open TX - */ - @Override - public Long getCurrentWriteId() { - if (currentTxnIndex >= 0) { - return txnToWriteIds.get(currentTxnIndex).getWriteId(); - } - return -1L; - } - - /** - * get state of current transaction - * @return - */ - @Override - public TxnState getCurrentTransactionState() { - return state; - } - - /** - * Remaining transactions are the ones that are not committed or aborted or active. - * Active transaction is not considered part of remaining txns. - * @return number of transactions remaining this batch. - */ - @Override - public int remainingTransactions() { - if (currentTxnIndex>=0) { - return txnToWriteIds.size() - currentTxnIndex -1; - } - return txnToWriteIds.size(); - } - - - /** - * Write record using RecordWriter - * @param record the data to be written - * @throws StreamingIOFailure I/O failure - * @throws SerializationError serialization error - * @throws ImpersonationFailed error writing on behalf of proxyUser - * @throws InterruptedException - */ - @Override - public void write(final byte[] record) - throws StreamingException, InterruptedException { - write(Collections.singletonList(record)); - } - private void checkIsClosed() throws IllegalStateException { - if(isClosed) { - throw new IllegalStateException("TransactionBatch " + toString() + " has been closed()"); - } - } - /** - * A transaction batch opens a single HDFS file and writes multiple transaction to it. If there is any issue - * with the write, we can't continue to write to the same file any as it may be corrupted now (at the tail). - * This ensures that a client can't ignore these failures and continue to write. - */ - private void markDead(boolean success) { - if(success) { - return; - } - isClosed = true;//also ensures that heartbeat() is no-op since client is likely doing it async - try { - abort(true);//abort all remaining txns - } - catch(Exception ex) { - LOG.error("Fatal error on " + toString() + "; cause " + ex.getMessage(), ex); - } - try { - closeImpl(); - } - catch (Exception ex) { - LOG.error("Fatal error on " + toString() + "; cause " + ex.getMessage(), ex); - } - } - - - /** - * Write records using RecordWriter - * @param records collection of rows to be written - * @throws StreamingException serialization error - * @throws ImpersonationFailed error writing on behalf of proxyUser - * @throws InterruptedException - */ - @Override - public void write(final Collection records) - throws StreamingException, InterruptedException, - ImpersonationFailed { - checkIsClosed(); - boolean success = false; - try { - if (ugi == null) { - writeImpl(records); - } else { - ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public Void run() throws StreamingException { - writeImpl(records); - return null; - } - } - ); - } - success = true; - } catch(SerializationError ex) { - //this exception indicates that a {@code record} could not be parsed and the - //caller can decide whether to drop it or send it to dead letter queue. - //rolling back the txn and retrying won't help since the tuple will be exactly the same - //when it's replayed. - success = true; - throw ex; - } catch(IOException e){ - throw new ImpersonationFailed("Failed writing as user '" + username + - "' to endPoint :" + endPt + ". Transaction Id: " - + getCurrentTxnId(), e); - } - finally { - markDead(success); - } - } - - private void writeImpl(Collection records) - throws StreamingException { - for (byte[] record : records) { - recordWriter.write(getCurrentWriteId(), record); - } - } - - - /** - * Commit the currently open transaction - * @throws TransactionError - * @throws StreamingIOFailure if flushing records failed - * @throws ImpersonationFailed if - * @throws InterruptedException - */ - @Override - public void commit() throws TransactionError, StreamingException, - ImpersonationFailed, InterruptedException { - checkIsClosed(); - boolean success = false; - try { - if (ugi == null) { - commitImpl(); - } - else { - ugi.doAs( - new PrivilegedExceptionAction() { - @Override - public Void run() throws StreamingException { - commitImpl(); - return null; - } - } - ); - } - success = true; - } catch (IOException e) { - throw new ImpersonationFailed("Failed committing Txn ID " + getCurrentTxnId() + " as user '" - + username + "'on endPoint :" + endPt + ". Transaction Id: ", e); - } - finally { - markDead(success); - } - } - - private void commitImpl() throws TransactionError, StreamingException { - try { - recordWriter.flush(); - msClient.commitTxn(txnToWriteIds.get(currentTxnIndex).getTxnId()); - state = TxnState.COMMITTED; - txnStatus[currentTxnIndex] = TxnState.COMMITTED; - } catch (NoSuchTxnException e) { - throw new TransactionError("Invalid transaction id : " - + getCurrentTxnId(), e); - } catch (TxnAbortedException e) { - throw new TransactionError("Aborted transaction cannot be committed" - , e); - } catch (TException e) { - throw new TransactionError("Unable to commit transaction" - + getCurrentTxnId(), e); - } - } - - /** - * Abort the currently open transaction - * @throws TransactionError - */ - @Override - public void abort() throws TransactionError, StreamingException - , ImpersonationFailed, InterruptedException { - if(isClosed) { - /** - * isDead is only set internally by this class. {@link #markDead(boolean)} will abort all - * remaining txns, so make this no-op to make sure that a well-behaved client that calls abort() - * error doesn't get misleading errors - */ - return; - } - abort(false); - } - private void abort(final boolean abortAllRemaining) throws TransactionError, StreamingException - , ImpersonationFailed, InterruptedException { - if (ugi==null) { - abortImpl(abortAllRemaining); - return; - } - try { - ugi.doAs ( - new PrivilegedExceptionAction() { - @Override - public Void run() throws StreamingException { - abortImpl(abortAllRemaining); - return null; - } - } - ); - } catch (IOException e) { - throw new ImpersonationFailed("Failed aborting Txn " + getCurrentTxnId() + " as user '" - + username + "' on endPoint :" + endPt, e); - } - } - - private void abortImpl(boolean abortAllRemaining) throws TransactionError, StreamingException { - try { - if(abortAllRemaining) { - //when last txn finished (abort/commit) the currentTxnIndex is pointing at that txn - //so we need to start from next one, if any. Also if batch was created but - //fetchTransactionBatch() was never called, we want to start with first txn - int minOpenTxnIndex = Math.max(currentTxnIndex + - (state == TxnState.ABORTED || state == TxnState.COMMITTED ? 1 : 0), 0); - for(currentTxnIndex = minOpenTxnIndex; - currentTxnIndex < txnToWriteIds.size(); currentTxnIndex++) { - msClient.rollbackTxn(txnToWriteIds.get(currentTxnIndex).getTxnId()); - txnStatus[currentTxnIndex] = TxnState.ABORTED; - } - currentTxnIndex--;//since the loop left it == txnToWriteIds.size() - } - else { - if (getCurrentTxnId() > 0) { - msClient.rollbackTxn(getCurrentTxnId()); - txnStatus[currentTxnIndex] = TxnState.ABORTED; - } - } - state = TxnState.ABORTED; - recordWriter.clear(); - } catch (NoSuchTxnException e) { - throw new TransactionError("Unable to abort invalid transaction id : " - + getCurrentTxnId(), e); - } catch (TException e) { - throw new TransactionError("Unable to abort transaction id : " - + getCurrentTxnId(), e); - } - } - - @Override - public void heartbeat() throws StreamingException, HeartBeatFailure { - if(isClosed) { - return; - } - if(state != TxnState.OPEN && currentTxnIndex >= txnToWriteIds.size() - 1) { - //here means last txn in the batch is resolved but the close() hasn't been called yet so - //there is nothing to heartbeat - return; - } - //if here after commit()/abort() but before next beginNextTransaction(), currentTxnIndex still - //points at the last txn which we don't want to heartbeat - Long first = txnToWriteIds.get(state == TxnState.OPEN ? currentTxnIndex : currentTxnIndex + 1).getTxnId(); - Long last = txnToWriteIds.get(txnToWriteIds.size()-1).getTxnId(); - try { - HeartbeatTxnRangeResponse resp = heartbeaterMSClient.heartbeatTxnRange(first, last); - if (!resp.getAborted().isEmpty() || !resp.getNosuch().isEmpty()) { - throw new HeartBeatFailure(resp.getAborted(), resp.getNosuch()); - } - } catch (TException e) { - throw new StreamingException("Failure to heartbeat on ids (" + first + "src/gen/thrift" - + last + ") on end point : " + endPt ); - } - } - - @Override - public boolean isClosed() { - return isClosed; - } - /** - * Close the TransactionBatch. This will abort any still open txns in this batch. - * @throws StreamingIOFailure I/O failure when closing transaction batch - */ - @Override - public void close() throws StreamingException, ImpersonationFailed, InterruptedException { - if(isClosed) { - return; - } - isClosed = true; - abortImpl(true);//abort proactively so that we don't wait for timeout - closeImpl();//perhaps we should add a version of RecordWriter.closeBatch(boolean abort) which - //will call RecordUpdater.close(boolean abort) - } - private void closeImpl() throws StreamingException, InterruptedException{ - state = TxnState.INACTIVE; - if(ugi == null) { - recordWriter.closeBatch(); - return; - } - try { - ugi.doAs ( - new PrivilegedExceptionAction() { - @Override - public Void run() throws StreamingException { - recordWriter.closeBatch(); - return null; - } - } - ); - try { - FileSystem.closeAllForUGI(ugi); - } catch (IOException exception) { - LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception); - } - } catch (IOException e) { - throw new ImpersonationFailed("Failed closing Txn Batch as user '" + username + - "' on endPoint :" + endPt, e); - } - } - - private static LockRequest createLockRequest(final HiveEndPoint hiveEndPoint, - String partNameForLock, String user, long txnId, String agentInfo) { - LockRequestBuilder rqstBuilder = agentInfo == null ? - new LockRequestBuilder() : new LockRequestBuilder(agentInfo); - rqstBuilder.setUser(user); - rqstBuilder.setTransactionId(txnId); - - LockComponentBuilder lockCompBuilder = new LockComponentBuilder() - .setDbName(hiveEndPoint.database) - .setTableName(hiveEndPoint.table) - .setSharedRead() - .setOperationType(DataOperationType.INSERT); - if (partNameForLock!=null && !partNameForLock.isEmpty() ) { - lockCompBuilder.setPartitionName(partNameForLock); - } - rqstBuilder.addLockComponent(lockCompBuilder.build()); - - return rqstBuilder.build(); - } - } // class TransactionBatchImpl - - static HiveConf createHiveConf(Class clazz, String metaStoreUri) { - HiveConf conf = new HiveConf(clazz); - if (metaStoreUri!= null) { - setHiveConf(conf, HiveConf.ConfVars.METASTOREURIS, metaStoreUri); - } - HiveEndPoint.overrideConfSettings(conf); - return conf; - } - - private static void overrideConfSettings(HiveConf conf) { - setHiveConf(conf, HiveConf.ConfVars.HIVE_TXN_MANAGER, - "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); - setHiveConf(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); - setHiveConf(conf, HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true); - // Avoids creating Tez Client sessions internally as it takes much longer currently - setHiveConf(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr"); - } - - private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, String value) { - if( LOG.isDebugEnabled() ) { - LOG.debug("Overriding HiveConf setting : " + var + " = " + value); - } - conf.setVar(var, value); - } - - private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, boolean value) { - if( LOG.isDebugEnabled() ) { - LOG.debug("Overriding HiveConf setting : " + var + " = " + value); - } - conf.setBoolVar(var, value); - } - -} // class HiveEndPoint diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ImpersonationFailed.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ImpersonationFailed.java deleted file mode 100644 index 7932077595..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/ImpersonationFailed.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class ImpersonationFailed extends StreamingException { - public ImpersonationFailed(String username, Exception e) { - super("Failed to impersonate user " + username, e); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidColumn.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidColumn.java deleted file mode 100644 index a7af608d76..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidColumn.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class InvalidColumn extends StreamingException { - - public InvalidColumn(String msg) { - super(msg); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidPartition.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidPartition.java deleted file mode 100644 index 82b6db8ddd..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidPartition.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class InvalidPartition extends StreamingException { - - public InvalidPartition(String partitionName, String partitionValue) { - super("Invalid partition: Name=" + partitionName + - ", Value=" + partitionValue); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTable.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTable.java deleted file mode 100644 index 9772c5c7bd..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTable.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class InvalidTable extends StreamingException { - - private static String makeMsg(String db, String table) { - return "Invalid table db:" + db + ", table:" + table; - } - - public InvalidTable(String db, String table) { - super(makeMsg(db,table), null); - } - - public InvalidTable(String db, String table, String msg) { - super(makeMsg(db, table) + ": " + msg, null); - } - - public InvalidTable(String db, String table, Exception inner) { - super(makeMsg(db, table) + ": " + inner.getMessage(), inner); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTrasactionState.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTrasactionState.java deleted file mode 100644 index 921d4dac9e..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/InvalidTrasactionState.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class InvalidTrasactionState extends TransactionError { - public InvalidTrasactionState(String msg) { - super(msg); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/PartitionCreationFailed.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/PartitionCreationFailed.java deleted file mode 100644 index 1913d33542..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/PartitionCreationFailed.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class PartitionCreationFailed extends StreamingException { - public PartitionCreationFailed(HiveEndPoint endPoint, Exception cause) { - super("Failed to create partition " + endPoint, cause); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/QueryFailedException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/QueryFailedException.java deleted file mode 100644 index f78be7f7ff..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/QueryFailedException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class QueryFailedException extends StreamingException { - String query; - - public QueryFailedException(String query, Exception e) { - super("Query failed: " + query + ". Due to :" + e.getMessage(), e); - this.query = query; - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java deleted file mode 100644 index 0f3c0bcfea..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -/** - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.RecordWriter - */ -@Deprecated -public interface RecordWriter { - - /** Writes using a hive RecordUpdater - * - * @param writeId the write ID of the table mapping to Txn in which the write occurs - * @param record the record to be written - */ - void write(long writeId, byte[] record) throws StreamingException; - - /** Flush records from buffer. Invoked by TransactionBatch.commit() */ - void flush() throws StreamingException; - - /** Clear bufferred writes. Invoked by TransactionBatch.abort() */ - void clear() throws StreamingException; - - /** Acquire a new RecordUpdater. Invoked when - * StreamingConnection.fetchTransactionBatch() is called */ - void newBatch(Long minWriteId, Long maxWriteID) throws StreamingException; - - /** Close the RecordUpdater. Invoked by TransactionBatch.close() */ - void closeBatch() throws StreamingException; -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/SerializationError.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/SerializationError.java deleted file mode 100644 index 33d2ceffe7..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/SerializationError.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -public class SerializationError extends StreamingException { - public SerializationError(String msg, Exception e) { - super(msg,e); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java deleted file mode 100644 index 3af9aed36b..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import org.apache.hadoop.security.UserGroupInformation; - -/** - * Represents a connection to a HiveEndPoint. Used to acquire transaction batches. - * Note: the expectation is that there is at most 1 TransactionBatch outstanding for any given - * StreamingConnection. Violating this may result in "out of sequence response". - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.HiveStreamingConnection - */ -@Deprecated -public interface StreamingConnection { - - /** - * Acquires a new batch of transactions from Hive. - - * @param numTransactionsHint is a hint from client indicating how many transactions client needs. - * @param writer Used to write record. The same writer instance can - * be shared with another TransactionBatch (to the same endpoint) - * only after the first TransactionBatch has been closed. - * Writer will be closed when the TransactionBatch is closed. - * @return - * @throws ConnectionError - * @throws InvalidPartition - * @throws StreamingException - * @return a batch of transactions - */ - public TransactionBatch fetchTransactionBatch(int numTransactionsHint, - RecordWriter writer) - throws ConnectionError, StreamingException, InterruptedException; - - /** - * Close connection - */ - public void close(); - - /** - * @return UserGroupInformation associated with this connection or {@code null} if there is none - */ - UserGroupInformation getUserGroupInformation(); -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingException.java deleted file mode 100644 index 421eaf079e..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class StreamingException extends Exception { - public StreamingException(String msg, Exception cause) { - super(msg, cause); - } - public StreamingException(String msg) { - super(msg); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingIOFailure.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingIOFailure.java deleted file mode 100644 index 247424f185..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingIOFailure.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -public class StreamingIOFailure extends StreamingException { - - public StreamingIOFailure(String msg, Exception cause) { - super(msg, cause); - } - - public StreamingIOFailure(String msg) { - super(msg); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java deleted file mode 100644 index d588f71a5c..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -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.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.io.Text; -import org.apache.hive.hcatalog.data.HCatRecordObjectInspector; -import org.apache.hive.hcatalog.data.JsonSerDe; - -import java.io.IOException; -import java.util.List; -import java.util.Properties; - -/** - * Streaming Writer handles utf8 encoded Json (Strict syntax). - * Uses org.apache.hive.hcatalog.data.JsonSerDe to process Json input - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.StrictJsonWriter - */ -@Deprecated -public class StrictJsonWriter extends AbstractRecordWriter { - private JsonSerDe serde; - - private final HCatRecordObjectInspector recordObjInspector; - private final ObjectInspector[] bucketObjInspectors; - private final StructField[] bucketStructFields; - - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #StrictJsonWriter(HiveEndPoint, HiveConf, StreamingConnection)} - */ - public StrictJsonWriter(HiveEndPoint endPoint) - throws ConnectionError, SerializationError, StreamingException { - this(endPoint, null, null); - } - - /** - * @deprecated As of release 1.3/2.1. Replaced by {@link #StrictJsonWriter(HiveEndPoint, HiveConf, StreamingConnection)} - */ - public StrictJsonWriter(HiveEndPoint endPoint, HiveConf conf) throws StreamingException { - this(endPoint, conf, null); - } - /** - * @param endPoint the end point to write to - * @throws ConnectionError - * @throws SerializationError - * @throws StreamingException - */ - public StrictJsonWriter(HiveEndPoint endPoint, StreamingConnection conn) - throws ConnectionError, SerializationError, StreamingException { - this(endPoint, null, conn); - } - /** - * @param endPoint the end point to write to - * @param conf a Hive conf object. Should be null if not using advanced Hive settings. - * @param conn connection this Writer is to be used with - * @throws ConnectionError - * @throws SerializationError - * @throws StreamingException - */ - public StrictJsonWriter(HiveEndPoint endPoint, HiveConf conf, StreamingConnection conn) - throws ConnectionError, SerializationError, StreamingException { - super(endPoint, conf, conn); - this.serde = createSerde(tbl, conf); - // get ObjInspectors for entire record and bucketed cols - try { - recordObjInspector = ( HCatRecordObjectInspector ) serde.getObjectInspector(); - this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, recordObjInspector); - } catch (SerDeException e) { - throw new SerializationError("Unable to get ObjectInspector for bucket columns", e); - } - - // get StructFields for bucketed cols - bucketStructFields = new StructField[bucketIds.size()]; - List allFields = recordObjInspector.getAllStructFieldRefs(); - for (int i = 0; i < bucketIds.size(); i++) { - bucketStructFields[i] = allFields.get(bucketIds.get(i)); - } - } - - @Override - public AbstractSerDe getSerde() { - return serde; - } - - protected HCatRecordObjectInspector getRecordObjectInspector() { - return recordObjInspector; - } - - @Override - protected StructField[] getBucketStructFields() { - return bucketStructFields; - } - - protected ObjectInspector[] getBucketObjectInspectors() { - return bucketObjInspectors; - } - - - @Override - public void write(long writeId, byte[] record) - throws StreamingIOFailure, SerializationError { - try { - Object encodedRow = encode(record); - int bucket = getBucket(encodedRow); - getRecordUpdater(bucket).insert(writeId, encodedRow); - } catch (IOException e) { - throw new StreamingIOFailure("Error writing record in transaction write id(" - + writeId + ")", e); - } - - } - - /** - * Creates JsonSerDe - * @param tbl used to create serde - * @param conf used to create serde - * @return - * @throws SerializationError if serde could not be initialized - */ - private static JsonSerDe createSerde(Table tbl, HiveConf conf) - throws SerializationError { - try { - Properties tableProps = MetaStoreUtils.getTableMetadata(tbl); - JsonSerDe serde = new JsonSerDe(); - SerDeUtils.initializeSerDe(serde, conf, tableProps, null); - return serde; - } catch (SerDeException e) { - throw new SerializationError("Error initializing serde " + JsonSerDe.class.getName(), e); - } - } - - @Override - public Object encode(byte[] utf8StrRecord) throws SerializationError { - try { - Text blob = new Text(utf8StrRecord); - return serde.deserialize(blob); - } catch (SerDeException e) { - throw new SerializationError("Unable to convert byte[] record into Object", e); - } - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java deleted file mode 100644 index 28406d38e8..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.RegexSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.Text; - -/** - * Streaming Writer handles text input data with regex. Uses - * org.apache.hadoop.hive.serde2.RegexSerDe - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.StrictRegexWriter - */ -@Deprecated -public class StrictRegexWriter extends AbstractRecordWriter { - private RegexSerDe serde; - private final StructObjectInspector recordObjInspector; - private final ObjectInspector[] bucketObjInspectors; - private final StructField[] bucketStructFields; - - /** - * @param endPoint the end point to write to - * @param conn connection this Writer is to be used with - * @throws ConnectionError - * @throws SerializationError - * @throws StreamingException - */ - public StrictRegexWriter(HiveEndPoint endPoint, StreamingConnection conn) - throws ConnectionError, SerializationError, StreamingException { - this(null, endPoint, null, conn); - } - - /** - * @param endPoint the end point to write to - * @param conf a Hive conf object. Should be null if not using advanced Hive settings. - * @param conn connection this Writer is to be used with - * @throws ConnectionError - * @throws SerializationError - * @throws StreamingException - */ - public StrictRegexWriter(HiveEndPoint endPoint, HiveConf conf, StreamingConnection conn) - throws ConnectionError, SerializationError, StreamingException { - this(null, endPoint, conf, conn); - } - - /** - * @param regex to parse the data - * @param endPoint the end point to write to - * @param conf a Hive conf object. Should be null if not using advanced Hive settings. - * @param conn connection this Writer is to be used with - * @throws ConnectionError - * @throws SerializationError - * @throws StreamingException - */ - public StrictRegexWriter(String regex, HiveEndPoint endPoint, HiveConf conf, StreamingConnection conn) - throws ConnectionError, SerializationError, StreamingException { - super(endPoint, conf, conn); - this.serde = createSerde(tbl, conf, regex); - // get ObjInspectors for entire record and bucketed cols - try { - recordObjInspector = ( StructObjectInspector ) serde.getObjectInspector(); - this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, recordObjInspector); - } catch (SerDeException e) { - throw new SerializationError("Unable to get ObjectInspector for bucket columns", e); - } - - // get StructFields for bucketed cols - bucketStructFields = new StructField[bucketIds.size()]; - List allFields = recordObjInspector.getAllStructFieldRefs(); - for (int i = 0; i < bucketIds.size(); i++) { - bucketStructFields[i] = allFields.get(bucketIds.get(i)); - } - } - - @Override - public AbstractSerDe getSerde() { - return serde; - } - - @Override - protected StructObjectInspector getRecordObjectInspector() { - return recordObjInspector; - } - - @Override - protected StructField[] getBucketStructFields() { - return bucketStructFields; - } - - @Override - protected ObjectInspector[] getBucketObjectInspectors() { - return bucketObjInspectors; - } - - - @Override - public void write(long writeId, byte[] record) - throws StreamingIOFailure, SerializationError { - try { - Object encodedRow = encode(record); - int bucket = getBucket(encodedRow); - getRecordUpdater(bucket).insert(writeId, encodedRow); - } catch (IOException e) { - throw new StreamingIOFailure("Error writing record in transaction write id(" - + writeId + ")", e); - } - } - - /** - * Creates RegexSerDe - * @param tbl used to create serde - * @param conf used to create serde - * @param regex used to create serde - * @return - * @throws SerializationError if serde could not be initialized - */ - private static RegexSerDe createSerde(Table tbl, HiveConf conf, String regex) - throws SerializationError { - try { - Properties tableProps = MetaStoreUtils.getTableMetadata(tbl); - tableProps.setProperty(RegexSerDe.INPUT_REGEX, regex); - ArrayList tableColumns = getCols(tbl); - tableProps.setProperty(serdeConstants.LIST_COLUMNS, StringUtils.join(tableColumns, ",")); - RegexSerDe serde = new RegexSerDe(); - SerDeUtils.initializeSerDe(serde, conf, tableProps, null); - return serde; - } catch (SerDeException e) { - throw new SerializationError("Error initializing serde " + RegexSerDe.class.getName(), e); - } - } - - private static ArrayList getCols(Table table) { - List cols = table.getSd().getCols(); - ArrayList colNames = new ArrayList(cols.size()); - for (FieldSchema col : cols) { - colNames.add(col.getName().toLowerCase()); - } - return colNames; - } - - /** - * Encode Utf8 encoded string bytes using RegexSerDe - * - * @param utf8StrRecord - * @return The encoded object - * @throws SerializationError - */ - @Override - public Object encode(byte[] utf8StrRecord) throws SerializationError { - try { - Text blob = new Text(utf8StrRecord); - return serde.deserialize(blob); - } catch (SerDeException e) { - throw new SerializationError("Unable to convert byte[] record into Object", e); - } - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java deleted file mode 100644 index 96aae02170..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - - -import java.util.Collection; - -/** - * Represents a set of Transactions returned by Hive. Supports opening, writing to - * and commiting/aborting each transaction. The interface is designed to ensure - * transactions in a batch are used up sequentially. To stream to the same HiveEndPoint - * concurrently, create separate StreamingConnections. - * - * Note on thread safety: At most 2 threads can run through a given TransactionBatch at the same - * time. One thread may call {@link #heartbeat()} and the other all other methods. - * Violating this may result in "out of sequence response". - * @deprecated as of Hive 3.0.0, replaced by org.apache.hive.streaming.HiveStreamingConnection - */ -@Deprecated -public interface TransactionBatch { - enum TxnState { - INACTIVE("I"), OPEN("O"), COMMITTED("C"), ABORTED("A"); - - private final String code; - TxnState(String code) { - this.code = code; - }; - public String toString() { - return code; - } - } - - /** - * Activate the next available transaction in the current transaction batch. - * @throws StreamingException if not able to switch to next Txn - * @throws InterruptedException if call in interrupted - */ - void beginNextTransaction() throws StreamingException, InterruptedException; - - /** - * Get Id of currently open transaction. - * @return transaction id - */ - Long getCurrentTxnId(); - - - /** - * Get write Id mapping to currently open transaction. - * @return write id - */ - Long getCurrentWriteId(); - - /** - * get state of current transaction. - */ - TxnState getCurrentTransactionState(); - - /** - * Commit the currently open transaction. - * @throws StreamingException if there are errors committing - * @throws InterruptedException if call in interrupted - */ - void commit() throws StreamingException, InterruptedException; - - /** - * Abort the currently open transaction. - * @throws StreamingException if there are errors - * @throws InterruptedException if call in interrupted - */ - void abort() throws StreamingException, InterruptedException; - - /** - * Remaining transactions are the ones that are not committed or aborted or open. - * Current open transaction is not considered part of remaining txns. - * @return number of transactions remaining this batch. - */ - int remainingTransactions(); - - - /** - * Write record using RecordWriter. - * @param record the data to be written - * @throws StreamingException if there are errors when writing - * @throws InterruptedException if call in interrupted - */ - void write(byte[] record) throws StreamingException, InterruptedException; - - /** - * Write records using RecordWriter. - * @throws StreamingException if there are errors when writing - * @throws InterruptedException if call in interrupted - */ - void write(Collection records) throws StreamingException, InterruptedException; - - - /** - * Issues a heartbeat to hive metastore on the current and remaining txn ids - * to keep them from expiring. - * @throws StreamingException if there are errors - */ - void heartbeat() throws StreamingException; - - /** - * Close the TransactionBatch. - * @throws StreamingException if there are errors closing batch - * @throws InterruptedException if call in interrupted - */ - void close() throws StreamingException, InterruptedException; - boolean isClosed(); -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatchUnAvailable.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatchUnAvailable.java deleted file mode 100644 index ae3587edc9..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatchUnAvailable.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class TransactionBatchUnAvailable extends StreamingException { - public TransactionBatchUnAvailable(HiveEndPoint ep, Exception e) { - super("Unable to acquire transaction batch on end point: " + ep, e); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionError.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionError.java deleted file mode 100644 index d438447b18..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionError.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -public class TransactionError extends StreamingException { - public TransactionError(String msg, Exception e) { - super(msg + (e == null ? "" : ": " + e.getMessage()), e); - } - - public TransactionError(String msg) { - super(msg); - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java deleted file mode 100644 index ebe032d705..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Creates/configures {@link HiveConf} instances with required ACID attributes. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class HiveConfFactory { - - private static final Logger LOG = LoggerFactory.getLogger(HiveConfFactory.class); - private static final String TRANSACTION_MANAGER = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; - - public static HiveConf newInstance(Configuration configuration, Class clazz, String metaStoreUri) { - HiveConf hiveConf = null; - if (configuration != null) { - if (!HiveConf.class.isAssignableFrom(configuration.getClass())) { - hiveConf = new HiveConf(configuration, clazz); - } else { - hiveConf = (HiveConf) configuration; - } - } - - if (hiveConf == null) { - hiveConf = HiveConfFactory.newInstance(clazz, metaStoreUri); - } else { - HiveConfFactory.overrideSettings(hiveConf); - } - return hiveConf; - } - - public static HiveConf newInstance(Class clazz, String metaStoreUri) { - HiveConf conf = new HiveConf(clazz); - if (metaStoreUri != null) { - setHiveConf(conf, HiveConf.ConfVars.METASTOREURIS, metaStoreUri); - } - overrideSettings(conf); - return conf; - } - - public static void overrideSettings(HiveConf conf) { - setHiveConf(conf, HiveConf.ConfVars.HIVE_TXN_MANAGER, TRANSACTION_MANAGER); - setHiveConf(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); - setHiveConf(conf, HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true); - // Avoids creating Tez Client sessions internally as it takes much longer currently - setHiveConf(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr"); - } - - private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, String value) { - if (LOG.isDebugEnabled()) { - LOG.debug("Overriding HiveConf setting : {} = {}", var, value); - } - conf.setVar(var, value); - } - - private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, boolean value) { - if (LOG.isDebugEnabled()) { - LOG.debug("Overriding HiveConf setting : {} = {}", var, value); - } - conf.setBoolVar(var, value); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java deleted file mode 100644 index 615fc1a751..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate; - -import java.io.IOException; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; -import java.lang.reflect.UndeclaredThrowableException; -import java.security.PrivilegedExceptionAction; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.thrift.TException; - -import com.google.common.reflect.AbstractInvocationHandler; - -/** - * Creates a proxied {@link IMetaStoreClient client} that wraps calls in a {@link PrivilegedExceptionAction} if the - * {@link UserGroupInformation} is specified. Invokes directly otherwise. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class UgiMetaStoreClientFactory { - - private static Set I_META_STORE_CLIENT_METHODS = getIMetaStoreClientMethods(); - - private final String metaStoreUri; - private final HiveConf conf; - private final boolean secureMode; - private final UserGroupInformation authenticatedUser; - private final String user; - - public UgiMetaStoreClientFactory(String metaStoreUri, HiveConf conf, UserGroupInformation authenticatedUser, - String user, boolean secureMode) { - this.metaStoreUri = metaStoreUri; - this.conf = conf; - this.authenticatedUser = authenticatedUser; - this.user = user; - this.secureMode = secureMode; - if (metaStoreUri != null) { - conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreUri); - } - if (secureMode) { - conf.setBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL, true); - } - } - - public IMetaStoreClient newInstance() throws MetaException { - return newInstance(new HiveMetaStoreClient(conf)); - } - - public IMetaStoreClient newInstance(IMetaStoreClient delegate) throws MetaException { - return createProxy(delegate, user, authenticatedUser); - } - - @Override - public String toString() { - return "UgiMetaStoreClientFactory [metaStoreUri=" + metaStoreUri + ", secureMode=" + secureMode - + ", authenticatedUser=" + authenticatedUser + ", user=" + user + "]"; - } - - private IMetaStoreClient createProxy(final IMetaStoreClient delegate, final String user, - final UserGroupInformation authenticatedUser) { - InvocationHandler handler = new AbstractInvocationHandler() { - - @Override - protected Object handleInvocation(Object proxy, final Method method, final Object[] args) throws Throwable { - try { - if (!I_META_STORE_CLIENT_METHODS.contains(method) || authenticatedUser == null) { - return method.invoke(delegate, args); - } - try { - return authenticatedUser.doAs(new PrivilegedExceptionAction() { - @Override - public Object run() throws Exception { - return method.invoke(delegate, args); - } - }); - } catch (IOException | InterruptedException e) { - throw new TException("PrivilegedExceptionAction failed as user '" + user + "'.", e); - } - } catch (UndeclaredThrowableException | InvocationTargetException e) { - throw e.getCause(); - } - } - }; - - ClassLoader classLoader = IMetaStoreClient.class.getClassLoader(); - Class[] interfaces = new Class[] { IMetaStoreClient.class }; - Object proxy = Proxy.newProxyInstance(classLoader, interfaces, handler); - return IMetaStoreClient.class.cast(proxy); - } - - private static Set getIMetaStoreClientMethods() { - return new HashSet<>(Arrays.asList(IMetaStoreClient.class.getDeclaredMethods())); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java deleted file mode 100644 index 40de497b08..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import java.io.Serializable; - -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; - -/** - * Describes an ACID table that can receive mutation events. Used to encode the information required by workers to write - * ACID events without requiring them to once more retrieve the data from the meta store db. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class AcidTable implements Serializable { - - private static final long serialVersionUID = 1L; - - private final String databaseName; - private final String tableName; - private final boolean createPartitions; - private final TableType tableType; - private long writeId; - - private Table table; - - AcidTable(String databaseName, String tableName, boolean createPartitions, TableType tableType) { - this.databaseName = databaseName; - this.tableName = tableName; - this.createPartitions = createPartitions; - this.tableType = tableType; - } - - /** - * Returns {@code 0} until such a time that a {@link Transaction} has been acquired (when - * {@link MutatorClient#newTransaction()} exits), at which point this will return the - * write id. - */ - public long getWriteId() { - return writeId; - } - - public String getDatabaseName() { - return databaseName; - } - - public String getTableName() { - return tableName; - } - - public boolean createPartitions() { - return createPartitions; - } - - /** - * Returns {@code null} until such a time that the table described by the {@link #getDatabaseName() database_name} - * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when - * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding - * {@link StorageDescriptor#getOutputFormat() OutputFormat}. - */ - public String getOutputFormatName() { - return table != null ? table.getSd().getOutputFormat() : null; - } - - /** - * Returns {@code 0} until such a time that the table described by the {@link #getDatabaseName() database_name} - * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when - * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding - * {@link StorageDescriptor#getNumBuckets() total bucket count}. - */ - public int getTotalBuckets() { - return table != null ? table.getSd().getNumBuckets() : 0; - } - - public TableType getTableType() { - return tableType; - } - - public String getQualifiedName() { - return (databaseName + "." + tableName).toUpperCase(); - } - - /** - * Returns {@code null} until such a time that the table described by the {@link #getDatabaseName() database_name} - * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when - * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding {@link Table}. - * Provided as a convenience to API users who may wish to gather further meta data regarding the table without - * connecting with the meta store once more. - */ - public Table getTable() { - return table; - } - - void setWriteId(long writeId) { - this.writeId = writeId; - } - - void setTable(Table table) { - if (!databaseName.equalsIgnoreCase(table.getDbName())) { - throw new IllegalArgumentException("Incorrect database name."); - } - if (!tableName.equalsIgnoreCase(table.getTableName())) { - throw new IllegalArgumentException("Incorrect table name."); - } - this.table = table; - } - - @Override - public String toString() { - return "AcidTable [databaseName=" + databaseName + ", tableName=" + tableName + ", createPartitions=" - + createPartitions + ", tableType=" + tableType + ", outputFormatName=" + getOutputFormatName() - + ", totalBuckets=" + getTotalBuckets() + ", writeId=" + writeId + "]"; - } - -} \ No newline at end of file diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java deleted file mode 100644 index 43ac527e79..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.nio.charset.Charset; - -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TDeserializer; -import org.apache.thrift.TException; -import org.apache.thrift.TSerializer; -import org.apache.thrift.protocol.TCompactProtocol; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Utility to serialize/deserialize {@link AcidTable AcidTables} into strings so that they can be easily transported as - * {@link Configuration} properties. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class AcidTableSerializer { - - private static final Logger LOG = LoggerFactory.getLogger(AcidTableSerializer.class); - - /* Allow for improved schemes. */ - private static final String PROLOG_V1 = "AcidTableV1:"; - - /** Returns a base 64 encoded representation of the supplied {@link AcidTable}. */ - public static String encode(AcidTable table) throws IOException { - DataOutputStream data = null; - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try { - data = new DataOutputStream(bytes); - data.writeUTF(table.getDatabaseName()); - data.writeUTF(table.getTableName()); - data.writeBoolean(table.createPartitions()); - if (table.getWriteId() <= 0) { - LOG.warn("Write ID <= 0. The recipient is probably expecting a table write ID."); - } - data.writeLong(table.getWriteId()); - data.writeByte(table.getTableType().getId()); - - Table metaTable = table.getTable(); - if (metaTable != null) { - byte[] thrift = new TSerializer(new TCompactProtocol.Factory()).serialize(metaTable); - data.writeInt(thrift.length); - data.write(thrift); - } else { - LOG.warn("Meta store table is null. The recipient is probably expecting an instance."); - data.writeInt(0); - } - } catch (TException e) { - throw new IOException("Error serializing meta store table.", e); - } finally { - data.close(); - } - - return PROLOG_V1 + new String(Base64.encodeBase64(bytes.toByteArray()), Charset.forName("UTF-8")); - } - - /** Returns the {@link AcidTable} instance decoded from a base 64 representation. */ - public static AcidTable decode(String encoded) throws IOException { - if (!encoded.startsWith(PROLOG_V1)) { - throw new IllegalStateException("Unsupported version."); - } - encoded = encoded.substring(PROLOG_V1.length()); - - byte[] decoded = Base64.decodeBase64(encoded); - AcidTable table = null; - try (DataInputStream in = new DataInputStream(new ByteArrayInputStream(decoded))) { - String databaseName = in.readUTF(); - String tableName = in.readUTF(); - boolean createPartitions = in.readBoolean(); - long writeId = in.readLong(); - TableType tableType = TableType.valueOf(in.readByte()); - int thriftLength = in.readInt(); - - table = new AcidTable(databaseName, tableName, createPartitions, tableType); - table.setWriteId(writeId); - - Table metaTable = null; - if (thriftLength > 0) { - metaTable = new Table(); - try { - byte[] thriftEncoded = new byte[thriftLength]; - in.readFully(thriftEncoded, 0, thriftLength); - new TDeserializer(new TCompactProtocol.Factory()).deserialize(metaTable, thriftEncoded); - table.setTable(metaTable); - } catch (TException e) { - throw new IOException("Error deserializing meta store table.", e); - } - } - } - return table; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java deleted file mode 100644 index 206a0ba54e..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -public class ClientException extends Exception { - - private static final long serialVersionUID = 1L; - - ClientException(String message, Throwable cause) { - super(message, cause); - } - - ClientException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java deleted file mode 100644 index 2b3b299f54..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -public class ConnectionException extends ClientException { - - private static final long serialVersionUID = 1L; - - ConnectionException(String message, Throwable cause) { - super(message, cause); - } - - ConnectionException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java deleted file mode 100644 index 11664f6a7d..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java +++ /dev/null @@ -1,183 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Responsible for orchestrating {@link Transaction Transactions} within which ACID table mutation events can occur. - * Typically this will be a large batch of delta operations. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class MutatorClient implements Closeable { - - private static final Logger LOG = LoggerFactory.getLogger(MutatorClient.class); - private static final String TRANSACTIONAL_PARAM_KEY = "transactional"; - - private final IMetaStoreClient metaStoreClient; - private final Lock.Options lockOptions; - private final List tables; - private boolean connected; - - MutatorClient(IMetaStoreClient metaStoreClient, HiveConf configuration, LockFailureListener lockFailureListener, - String user, Collection tables) { - this.metaStoreClient = metaStoreClient; - this.tables = Collections.unmodifiableList(new ArrayList<>(tables)); - - lockOptions = new Lock.Options() - .configuration(configuration) - .lockFailureListener(lockFailureListener == null ? LockFailureListener.NULL_LISTENER : lockFailureListener) - .user(user); - for (AcidTable table : tables) { - switch (table.getTableType()) { - case SOURCE: - lockOptions.addSourceTable(table.getDatabaseName(), table.getTableName()); - break; - case SINK: - lockOptions.addSinkTable(table.getDatabaseName(), table.getTableName()); - break; - default: - throw new IllegalArgumentException("Unknown TableType: " + table.getTableType()); - } - } - } - - /** - * Connects to the {@link IMetaStoreClient meta store} that will be used to manage {@link Transaction} life-cycles. - * Also checks that the tables destined to receive mutation events are able to do so. The client should only hold one - * open transaction at any given time (TODO: enforce this). - */ - public void connect() throws ConnectionException { - if (connected) { - throw new ConnectionException("Already connected."); - } - for (AcidTable table : tables) { - checkTable(metaStoreClient, table); - } - LOG.debug("Connected to end point {}", metaStoreClient); - connected = true; - } - - /** Creates a new {@link Transaction} by opening a transaction with the {@link IMetaStoreClient meta store}. */ - public Transaction newTransaction() throws TransactionException { - if (!connected) { - throw new TransactionException("Not connected - cannot create transaction."); - } - Transaction transaction = new Transaction(metaStoreClient, lockOptions); - long txnId = transaction.getTransactionId(); - for (AcidTable table : tables) { - try { - table.setWriteId(metaStoreClient.allocateTableWriteId(txnId, - table.getDatabaseName(), table.getTableName())); - } catch (TException ex) { - try { - metaStoreClient.rollbackTxn(txnId); - } catch (TException e) { - LOG.warn("Allocation of write id failed for table {} and rollback transaction {} failed due to {}", - AcidUtils.getFullTableName(table.getDatabaseName(), table.getTableName()), txnId, e.getMessage()); - } - throw new TransactionException("Unable to allocate table write ID for table " - + AcidUtils.getFullTableName(table.getDatabaseName(), table.getTableName()) - + " under txn " + txnId, ex); - } - } - LOG.debug("Created transaction {}", transaction); - return transaction; - } - - /** Did the client connect successfully. Note the the client may have since become disconnected. */ - public boolean isConnected() { - return connected; - } - - /** - * Closes the client releasing any {@link IMetaStoreClient meta store} connections held. Does not notify any open - * transactions (TODO: perhaps it should?) - */ - @Override - public void close() throws IOException { - metaStoreClient.close(); - LOG.debug("Closed client."); - connected = false; - } - - /** - * Returns the list of managed {@link AcidTable AcidTables} that can receive mutation events under the control of this - * client. - */ - public List getTables() throws ConnectionException { - if (!connected) { - throw new ConnectionException("Not connected - cannot interrogate tables."); - } - return Collections. unmodifiableList(tables); - } - - @Override - public String toString() { - return "MutatorClient [metaStoreClient=" + metaStoreClient + ", connected=" + connected + "]"; - } - - private void checkTable(IMetaStoreClient metaStoreClient, AcidTable acidTable) throws ConnectionException { - try { - LOG.debug("Checking table {}.", acidTable.getQualifiedName()); - Table metaStoreTable = metaStoreClient.getTable(acidTable.getDatabaseName(), acidTable.getTableName()); - - if (acidTable.getTableType() == TableType.SINK) { - Map parameters = metaStoreTable.getParameters(); - if (!Boolean.parseBoolean(parameters.get(TRANSACTIONAL_PARAM_KEY))) { - throw new ConnectionException("Cannot stream to table that is not transactional: '" - + acidTable.getQualifiedName() + "'."); - } - int totalBuckets = metaStoreTable.getSd().getNumBuckets(); - LOG.debug("Table {} has {} buckets.", acidTable.getQualifiedName(), totalBuckets); - if (totalBuckets <= 0) { - throw new ConnectionException("Cannot stream to table that has not been bucketed: '" - + acidTable.getQualifiedName() + "'."); - } - - String outputFormat = metaStoreTable.getSd().getOutputFormat(); - LOG.debug("Table {} has {} OutputFormat.", acidTable.getQualifiedName(), outputFormat); - acidTable.setTable(metaStoreTable); - } - } catch (NoSuchObjectException e) { - throw new ConnectionException("Invalid table '" + acidTable.getQualifiedName() + "'", e); - } catch (TException e) { - throw new ConnectionException("Error communicating with the meta store", e); - } - LOG.debug("Table {} OK.", acidTable.getQualifiedName()); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java deleted file mode 100644 index 1575d8d4a4..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.hcatalog.common.HCatUtil; -import org.apache.hive.hcatalog.streaming.mutate.HiveConfFactory; -import org.apache.hive.hcatalog.streaming.mutate.UgiMetaStoreClientFactory; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener; - -/** Convenience class for building {@link MutatorClient} instances. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class MutatorClientBuilder { - - private final Map tables = new HashMap<>(); - private HiveConf configuration; - private UserGroupInformation authenticatedUser; - private String metaStoreUri; - public LockFailureListener lockFailureListener; - - public MutatorClientBuilder configuration(HiveConf conf) { - this.configuration = conf; - return this; - } - - public MutatorClientBuilder authenticatedUser(UserGroupInformation authenticatedUser) { - this.authenticatedUser = authenticatedUser; - return this; - } - - public MutatorClientBuilder metaStoreUri(String metaStoreUri) { - this.metaStoreUri = metaStoreUri; - return this; - } - - /** Set a listener to handle {@link Lock} failure events - highly recommended. */ - public MutatorClientBuilder lockFailureListener(LockFailureListener lockFailureListener) { - this.lockFailureListener = lockFailureListener; - return this; - } - - /** - * Adds a mutation event destination (an ACID table) to be managed by this client, which is either unpartitioned or - * will is not to have partitions created automatically. - */ - public MutatorClientBuilder addSourceTable(String databaseName, String tableName) { - addTable(databaseName, tableName, false, TableType.SOURCE); - return this; - } - - /** - * Adds a mutation event destination (an ACID table) to be managed by this client, which is either unpartitioned or - * will is not to have partitions created automatically. - */ - public MutatorClientBuilder addSinkTable(String databaseName, String tableName) { - return addSinkTable(databaseName, tableName, false); - } - - /** - * Adds a partitioned mutation event destination (an ACID table) to be managed by this client, where new partitions - * will be created as needed. - */ - public MutatorClientBuilder addSinkTable(String databaseName, String tableName, boolean createPartitions) { - addTable(databaseName, tableName, createPartitions, TableType.SINK); - return this; - } - - private void addTable(String databaseName, String tableName, boolean createPartitions, TableType tableType) { - if (databaseName == null) { - throw new IllegalArgumentException("Database cannot be null"); - } - if (tableName == null) { - throw new IllegalArgumentException("Table cannot be null"); - } - String key = (databaseName + "." + tableName).toUpperCase(); - AcidTable previous = tables.get(key); - if (previous != null) { - if (tableType == TableType.SINK && previous.getTableType() != TableType.SINK) { - tables.remove(key); - } else { - throw new IllegalArgumentException("Table has already been added: " + databaseName + "." + tableName); - } - } - - Table table = new Table(); - table.setDbName(databaseName); - table.setTableName(tableName); - tables.put(key, new AcidTable(databaseName, tableName, createPartitions, tableType)); - } - - /** Builds the client. */ - public MutatorClient build() throws ClientException, MetaException { - String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName(); - boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials(); - - configuration = HiveConfFactory.newInstance(configuration, this.getClass(), metaStoreUri); - - IMetaStoreClient metaStoreClient; - try { - metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser, user, secureMode) - .newInstance(HCatUtil.getHiveMetastoreClient(configuration)); - } catch (IOException e) { - throw new ClientException("Could not create meta store client.", e); - } - - return new MutatorClient(metaStoreClient, configuration, lockFailureListener, user, tables.values()); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java deleted file mode 100644 index 02c9e69605..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -public enum TableType { - SOURCE((byte) 0), - SINK((byte) 1); - - private static final TableType[] INDEX = buildIndex(); - - private static TableType[] buildIndex() { - TableType[] index = new TableType[TableType.values().length]; - for (TableType type : values()) { - byte position = type.getId(); - if (index[position] != null) { - throw new IllegalStateException("Overloaded index: " + position); - } - index[position] = type; - } - return index; - } - - private byte id; - - private TableType(byte id) { - this.id = id; - } - - public byte getId() { - return id; - } - - public static TableType valueOf(byte id) { - if (id < 0 || id >= INDEX.length) { - throw new IllegalArgumentException("Invalid id: " + id); - } - return INDEX[id]; - } -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java deleted file mode 100644 index e1c6735d6d..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockException; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class Transaction { - - private static final Logger LOG = LoggerFactory.getLogger(Transaction.class); - - private final Lock lock; - private final IMetaStoreClient metaStoreClient; - private final long transactionId; - - private TxnState state; - - Transaction(IMetaStoreClient metaStoreClient, Lock.Options lockOptions) throws TransactionException { - this(metaStoreClient, new Lock(metaStoreClient, lockOptions)); - } - - /** Visible for testing only. */ - Transaction(IMetaStoreClient metaStoreClient, Lock lock) throws TransactionException { - this.metaStoreClient = metaStoreClient; - this.lock = lock; - transactionId = open(lock.getUser()); - } - - public long getTransactionId() { - return transactionId; - } - - public TxnState getState() { - return state; - } - - /** - * Begin the transaction. Acquires a {@link Lock} for the transaction and {@link AcidTable AcidTables}. - */ - public void begin() throws TransactionException { - try { - lock.acquire(transactionId); - } catch (LockException e) { - throw new TransactionException("Unable to acquire lock for transaction: " + transactionId, e); - } - state = TxnState.OPEN; - LOG.debug("Begin. Transaction id: {}", transactionId); - } - - /** Commits the transaction. Releases the {@link Lock}. */ - public void commit() throws TransactionException { - try { - lock.release(); - } catch (LockException e) { - // This appears to leave the remove transaction in an inconsistent state but the heartbeat is now - // cancelled and it will eventually time out - throw new TransactionException("Unable to release lock: " + lock + " for transaction: " + transactionId, e); - } - try { - metaStoreClient.commitTxn(transactionId); - state = TxnState.COMMITTED; - } catch (NoSuchTxnException e) { - throw new TransactionException("Invalid transaction id: " + transactionId, e); - } catch (TxnAbortedException e) { - throw new TransactionException("Aborted transaction cannot be committed: " + transactionId, e); - } catch (TException e) { - throw new TransactionException("Unable to commit transaction: " + transactionId, e); - } - LOG.debug("Committed. Transaction id: {}", transactionId); - } - - /** Aborts the transaction. Releases the {@link Lock}. */ - public void abort() throws TransactionException { - try { - lock.release(); - } catch (LockException e) { - // This appears to leave the remove transaction in an inconsistent state but the heartbeat is now - // cancelled and it will eventually time out - throw new TransactionException("Unable to release lock: " + lock + " for transaction: " + transactionId, e); - } - try { - metaStoreClient.rollbackTxn(transactionId); - state = TxnState.ABORTED; - } catch (NoSuchTxnException e) { - throw new TransactionException("Unable to abort invalid transaction id : " + transactionId, e); - } catch (TException e) { - throw new TransactionException("Unable to abort transaction id : " + transactionId, e); - } - LOG.debug("Aborted. Transaction id: {}", transactionId); - } - - @Override - public String toString() { - return "Transaction [transactionId=" + transactionId + ", state=" + state + "]"; - } - - private long open(String user) throws TransactionException { - long transactionId = -1; - try { - transactionId = metaStoreClient.openTxn(user); - state = TxnState.INACTIVE; - } catch (TException e) { - throw new TransactionException("Unable to open transaction for user: " + user, e); - } - LOG.debug("Opened transaction with id: {}", transactionId); - return transactionId; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java deleted file mode 100644 index 21cffa12a9..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -public class TransactionException extends ClientException { - - private static final long serialVersionUID = 1L; - - TransactionException(String message, Throwable cause) { - super(message, cause); - } - - TransactionException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java deleted file mode 100644 index ba0fa1e149..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import java.util.Collection; -import java.util.Timer; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Table; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Creates a default {@link HeartbeatTimerTask} for {@link Lock Locks}. */ -class HeartbeatFactory { - - private static final Logger LOG = LoggerFactory.getLogger(HeartbeatFactory.class); - - /** Creates a new {@link HeartbeatTimerTask} instance for the {@link Lock} and schedules it. */ - Timer newInstance(IMetaStoreClient metaStoreClient, LockFailureListener listener, Long transactionId, - Collection tableDescriptors, long lockId, int heartbeatPeriod) { - Timer heartbeatTimer = new Timer("hive-lock-heartbeat[lockId=" + lockId + ", transactionId=" + transactionId + "]", - true); - HeartbeatTimerTask task = new HeartbeatTimerTask(metaStoreClient, listener, transactionId, tableDescriptors, lockId); - heartbeatTimer.schedule(task, TimeUnit.SECONDS.toMillis(heartbeatPeriod), - TimeUnit.SECONDS.toMillis(heartbeatPeriod)); - - LOG.debug("Scheduled heartbeat timer task: {}", heartbeatTimer); - return heartbeatTimer; - } - -} \ No newline at end of file diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java deleted file mode 100644 index 81f99de390..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import java.util.Collection; -import java.util.TimerTask; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.NoSuchLockException; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.hive.hcatalog.streaming.mutate.client.Transaction; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * {@link TimerTask} that sends {@link IMetaStoreClient#heartbeat(long, long) heartbeat} events to the - * {@link IMetaStoreClient meta store} to keet the {@link Lock} and {@link Transaction} alive. Nofifies the registered - * {@link LockFailureListener} should the lock fail. - */ -class HeartbeatTimerTask extends TimerTask { - - private static final Logger LOG = LoggerFactory.getLogger(HeartbeatTimerTask.class); - - private final IMetaStoreClient metaStoreClient; - private final long lockId; - private final Long transactionId; - private final LockFailureListener listener; - private final Collection
tableDescriptors; - - HeartbeatTimerTask(IMetaStoreClient metaStoreClient, LockFailureListener listener, Long transactionId, - Collection
tableDescriptors, long lockId) { - this.metaStoreClient = metaStoreClient; - this.listener = listener; - this.transactionId = transactionId; - this.tableDescriptors = tableDescriptors; - this.lockId = lockId; - LOG.debug("Reporting to listener {}", listener); - } - - @Override - public void run() { - try { - // I'm assuming that there is no transaction ID for a read lock. - metaStoreClient.heartbeat(transactionId == null ? 0 : transactionId, lockId); - LOG.debug("Sent heartbeat for lock={}, transactionId={}", lockId, transactionId); - } catch (NoSuchLockException | NoSuchTxnException | TxnAbortedException e) { - failLock(e); - } catch (TException e) { - LOG.warn("Failed to send heartbeat to meta store.", e); - } - } - - private void failLock(Exception e) { - LOG.debug("Lock " + lockId + " failed, cancelling heartbeat and notifying listener: " + listener, e); - // Cancel the heartbeat - cancel(); - listener.lockFailed(lockId, transactionId, Lock.asStrings(tableDescriptors), e); - } - - @Override - public String toString() { - return "HeartbeatTimerTask [lockId=" + lockId + ", transactionId=" + transactionId + "]"; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java deleted file mode 100644 index 88970da3a5..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java +++ /dev/null @@ -1,327 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Set; -import java.util.Timer; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.LockComponentBuilder; -import org.apache.hadoop.hive.metastore.LockRequestBuilder; -import org.apache.hadoop.hive.metastore.api.DataOperationType; -import org.apache.hadoop.hive.metastore.api.LockComponent; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockState; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Manages the state required to safely read/write from/to an ACID table. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class Lock { - - private static final Logger LOG = LoggerFactory.getLogger(Lock.class); - - private static final double HEARTBEAT_FACTOR = 0.75; - private static final int DEFAULT_HEARTBEAT_PERIOD = 275; - - private final IMetaStoreClient metaStoreClient; - private final HeartbeatFactory heartbeatFactory; - private final LockFailureListener listener; - private final Collection
sinks; - private final Collection
tables = new HashSet<>(); - private final int lockRetries; - private final int retryWaitSeconds; - private final String user; - private final HiveConf hiveConf; - - private Timer heartbeat; - private Long lockId; - private Long transactionId; - - public Lock(IMetaStoreClient metaStoreClient, Options options) { - this(metaStoreClient, new HeartbeatFactory(), options.hiveConf, options.listener, options.user, options.sources, - options.sinks, options.lockRetries, options.retryWaitSeconds); - } - - /** Visible for testing only. */ - Lock(IMetaStoreClient metaStoreClient, HeartbeatFactory heartbeatFactory, HiveConf hiveConf, - LockFailureListener listener, String user, Collection
sources, Collection
sinks, int lockRetries, - int retryWaitSeconds) { - this.metaStoreClient = metaStoreClient; - this.heartbeatFactory = heartbeatFactory; - this.hiveConf = hiveConf; - this.user = user; - this.listener = listener; - this.lockRetries = lockRetries; - this.retryWaitSeconds = retryWaitSeconds; - - this.sinks = sinks; - tables.addAll(sources); - tables.addAll(sinks); - - if (LockFailureListener.NULL_LISTENER.equals(listener)) { - LOG.warn("No {} supplied. Data quality and availability cannot be assured.", - LockFailureListener.class.getSimpleName()); - } - } - - /** Attempts to acquire a read lock on the table, returns if successful, throws exception otherwise. */ - public void acquire() throws LockException { - lockId = internalAcquire(null); - initiateHeartbeat(); - } - - /** Attempts to acquire a read lock on the table, returns if successful, throws exception otherwise. */ - public void acquire(long transactionId) throws LockException { - if (transactionId <= 0) { - throw new IllegalArgumentException("Invalid transaction id: " + transactionId); - } - lockId = internalAcquire(transactionId); - this.transactionId = transactionId; - initiateHeartbeat(); - } - - /** Attempts to release the read lock on the table. Throws an exception if the lock failed at any point. */ - public void release() throws LockException { - if (heartbeat != null) { - heartbeat.cancel(); - } - internalRelease(); - } - - public String getUser() { - return user; - } - - @Override - public String toString() { - return "Lock [metaStoreClient=" + metaStoreClient + ", lockId=" + lockId + ", transactionId=" + transactionId + "]"; - } - - private long internalAcquire(Long transactionId) throws LockException { - int attempts = 0; - LockRequest request = buildLockRequest(transactionId); - do { - LockResponse response = null; - try { - response = metaStoreClient.lock(request); - } catch (TException e) { - throw new LockException("Unable to acquire lock for tables: [" + join(tables) + "]", e); - } - if (response != null) { - LockState state = response.getState(); - if (state == LockState.NOT_ACQUIRED || state == LockState.ABORT) { - // I expect we'll only see NOT_ACQUIRED here? - break; - } - if (state == LockState.ACQUIRED) { - LOG.debug("Acquired lock {}", response.getLockid()); - return response.getLockid(); - } - if (state == LockState.WAITING) { - try { - Thread.sleep(TimeUnit.SECONDS.toMillis(retryWaitSeconds)); - } catch (InterruptedException e) { - } - } - } - attempts++; - } while (attempts < lockRetries); - throw new LockException("Could not acquire lock on tables: [" + join(tables) + "]"); - } - - private void internalRelease() { - try { - // if there is a transaction then this lock will be released on commit/abort/rollback instead. - if (lockId != null && transactionId == null) { - metaStoreClient.unlock(lockId); - LOG.debug("Released lock {}", lockId); - lockId = null; - } - } catch (TException e) { - LOG.error("Lock " + lockId + " failed.", e); - listener.lockFailed(lockId, transactionId, asStrings(tables), e); - } - } - - private LockRequest buildLockRequest(Long transactionId) { - if (transactionId == null && !sinks.isEmpty()) { - throw new IllegalArgumentException("Cannot sink to tables outside of a transaction: sinks=" + asStrings(sinks)); - } - LockRequestBuilder requestBuilder = new LockRequestBuilder(); - for (Table table : tables) { - LockComponentBuilder componentBuilder = new LockComponentBuilder().setDbName(table.getDbName()).setTableName( - table.getTableName()); - //todo: DataOperationType is set conservatively here, we'd really want to distinguish update/delete - //and insert/select and if resource (that is written to) is ACID or not - if (sinks.contains(table)) { - componentBuilder.setSharedWrite().setOperationType(DataOperationType.UPDATE).setIsTransactional(true); - } else { - componentBuilder.setSharedRead().setOperationType(DataOperationType.INSERT).setIsTransactional(true); - } - LockComponent component = componentBuilder.build(); - requestBuilder.addLockComponent(component); - } - if (transactionId != null) { - requestBuilder.setTransactionId(transactionId); - } - LockRequest request = requestBuilder.setUser(user).build(); - return request; - } - - private void initiateHeartbeat() { - int heartbeatPeriod = getHeartbeatPeriod(); - LOG.debug("Heartbeat period {}s", heartbeatPeriod); - heartbeat = heartbeatFactory.newInstance(metaStoreClient, listener, transactionId, tables, lockId, heartbeatPeriod); - } - - private int getHeartbeatPeriod() { - int heartbeatPeriod = DEFAULT_HEARTBEAT_PERIOD; - if (hiveConf != null) { - // This value is always in seconds and includes an 's' suffix. - String txTimeoutSeconds = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT); - if (txTimeoutSeconds != null) { - // We want to send the heartbeat at an interval that is less than the timeout. - heartbeatPeriod = Math.max(1, - (int) (Integer.parseInt(txTimeoutSeconds.substring(0, txTimeoutSeconds.length() - 1)) * HEARTBEAT_FACTOR)); - } - } - return heartbeatPeriod; - } - - /** Visible for testing only. */ - Long getLockId() { - return lockId; - } - - /** Visible for testing only. */ - Long getTransactionId() { - return transactionId; - } - - /** Visible for testing only. */ - static String join(Iterable values) { - return StringUtils.join(values, ","); - } - - /** Visible for testing only. */ - static List asStrings(Collection
tables) { - List strings = new ArrayList<>(tables.size()); - for (Table descriptor : tables) { - strings.add(descriptor.getDbName() + "." + descriptor.getTableName()); - } - return strings; - } - - /** Constructs a lock options for a set of Hive ACID tables from which we wish to read. */ - public static final class Options { - Set
sources = new LinkedHashSet<>(); - Set
sinks = new LinkedHashSet<>(); - LockFailureListener listener = LockFailureListener.NULL_LISTENER; - int lockRetries = 5; - int retryWaitSeconds = 30; - String user; - HiveConf hiveConf; - - /** Adds a table for which a shared lock will be requested. */ - public Options addSourceTable(String databaseName, String tableName) { - addTable(databaseName, tableName, sources); - return this; - } - - /** Adds a table for which a semi-shared lock will be requested. */ - public Options addSinkTable(String databaseName, String tableName) { - addTable(databaseName, tableName, sinks); - return this; - } - - private void addTable(String databaseName, String tableName, Set
tables) { - checkNotNullOrEmpty(databaseName); - checkNotNullOrEmpty(tableName); - Table table = new Table(); - table.setDbName(databaseName); - table.setTableName(tableName); - tables.add(table); - } - - public Options user(String user) { - checkNotNullOrEmpty(user); - this.user = user; - return this; - } - - public Options configuration(HiveConf hiveConf) { - checkNotNull(hiveConf); - this.hiveConf = hiveConf; - return this; - } - - /** Sets a listener to handle failures of locks that were previously acquired. */ - public Options lockFailureListener(LockFailureListener listener) { - checkNotNull(listener); - this.listener = listener; - return this; - } - - public Options lockRetries(int lockRetries) { - checkArgument(lockRetries > 0); - this.lockRetries = lockRetries; - return this; - } - - public Options retryWaitSeconds(int retryWaitSeconds) { - checkArgument(retryWaitSeconds > 0); - this.retryWaitSeconds = retryWaitSeconds; - return this; - } - - private static void checkArgument(boolean value) { - if (!value) { - throw new IllegalArgumentException(); - } - } - - private static void checkNotNull(Object value) { - if (value == null) { - throw new IllegalArgumentException(); - } - } - - private static void checkNotNullOrEmpty(String value) { - if (StringUtils.isBlank(value)) { - throw new IllegalArgumentException(); - } - } - - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java deleted file mode 100644 index bce232a883..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -public class LockException extends Exception { - - private static final long serialVersionUID = 1L; - - public LockException(String message) { - super(message); - } - - public LockException(String message, Throwable cause) { - super(message, cause); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java deleted file mode 100644 index a3845ea784..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Provides a means to handle the situation when a held lock fails. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public interface LockFailureListener { - - static final Logger LOG = LoggerFactory.getLogger(LockFailureListener.class); - - static final LockFailureListener NULL_LISTENER = new LockFailureListener() { - @Override - public void lockFailed(long lockId, Long transactionId, Iterable tableNames, Throwable t) { - LOG.warn( - "Ignored lock failure: lockId=" + lockId + ", transactionId=" + transactionId + ", tables=" + tableNames, t); - } - - public String toString() { - return LockFailureListener.class.getName() + ".NULL_LISTENER"; - } - }; - - /** Called when the specified lock has failed. You should probably abort your job in this case. */ - void lockFailed(long lockId, Long transactionId, Iterable tableNames, Throwable t); - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot deleted file mode 100644 index c5a8dbdf1c..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -digraph "API Usage" { - nodesep=1.2; - - DATA [label="ACID\ndataset",shape=oval,style=filled,color="gray"]; - CHANGES [label="Changed\ndata",shape=oval,style=filled,color="gray"]; - - META_STORE [label="Hive\nMetaStore",shape=box,style=filled,color="darkseagreen3"]; - HIVE_CLI [label="Hive\nCLI",shape=box,style=filled,color="darkseagreen3"]; - - MERGE1 [label="Compute\nmutations\n(your code)",shape=box,style=filled,color="khaki1"]; - SORT [label="Group\n& sort\n(your code)",shape=box,style=filled,color="khaki1"]; - CLIENT [label="Mutator\nclient",shape=box,style=filled,color="lightblue"]; - BUCKET [label="Bucket ID\nappender",shape=box,style=filled,color="lightblue"]; - COORD [label="Mutator\ncoordinator",shape=box,style=filled,color="lightblue"]; - CLIENT -> COORD [label="Provides\nconf to"]; - CLIENT -> BUCKET [label="Provides\nconf to"]; - - CLIENT -> META_STORE [label="Manages\ntxns using"]; - CHANGES -> MERGE1 [label="Reads ∆s\nfrom"]; - DATA -> MERGE1 [label="Reads\nROW__IDs\nfrom"]; - BUCKET -> MERGE1 [label="Appends ids\nto inserts"]; - MERGE1 -> SORT; - SORT -> COORD [label="Issues\nmutations to"]; - COORD -> DATA [label="Writes to"]; - DATA -> HIVE_CLI [label="Read by"]; - META_STORE -> DATA [label="Compacts"]; -} \ No newline at end of file diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html deleted file mode 100644 index 7bc75c0ee0..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html +++ /dev/null @@ -1,520 +0,0 @@ - - - - - - - - -HCatalog Streaming Mutation API - - - - -

HCatalog Streaming Mutation API -- high level description

-

@deprecated as of Hive 3.0.0

-

Background

-

-In certain data processing use cases it is necessary to modify existing -data when new facts arrive. An example of this is the classic ETL merge -where a copy of a data set is kept in sync with a master by the frequent -application of deltas. The deltas describe the mutations (inserts, -updates, deletes) that have occurred to the master since the previous -sync. To implement such a case using Hadoop traditionally demands that -the partitions containing records targeted by the mutations be -rewritten. This is a coarse approach; a partition containing millions of -records might be rebuilt because of a single record change. Additionally -these partitions cannot be restated atomically; at some point the old -partition data must be swapped with the new partition data. When this -swap occurs, usually by issuing an HDFS -rm -followed by a -mv -, the possibility exists where the data appears to be unavailable and -hence any downstream jobs consuming the data might unexpectedly fail. -Therefore data processing patterns that restate raw data on HDFS cannot -operate robustly without some external mechanism to orchestrate -concurrent access to changing data. -

- -

-The availability of ACID tables in Hive provides a mechanism that both -enables concurrent access to data stored in HDFS (so long as it's in the -ORC+ACID format), and also permits row level mutations or records within -a table, without the need to rewrite the existing data. But while Hive -itself supports -INSERT -, -UPDATE -and -DELETE -commands, and the ORC format can support large batches of mutations in a -transaction, Hive's execution engine currently submits each individual -mutation operation in a separate transaction and issues table scans (M/R -jobs) to execute them. It does not currently scale to the demands of -processing large deltas in an atomic manner. Furthermore it would be -advantageous to extend atomic batch mutation capabilities beyond Hive by -making them available to other data processing frameworks. The Streaming -Mutation API does just this. -

- -

The Streaming Mutation API, although similar to the Streaming -API, has a number of differences and are built to enable very different -use cases. Superficially, the Streaming API can only write new data -whereas the mutation API can also modify existing data. However the two -APIs also based on very different transaction models. The Streaming API -focuses on surfacing a continuous stream of new data into a Hive table -and does so by batching small sets of writes into multiple short-lived -transactions. Conversely the mutation API is designed to infrequently -apply large sets of mutations to a data set in an atomic fashion; all -mutations will either be applied or they will not. This instead mandates -the use of a single long-lived transaction. This table summarises the -attributes of each API:

- -
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
AttributeStreaming APIMutation API
Ingest typeData arrives continuouslyIngests are performed periodically and the mutations are -applied in a single batch
Transaction scopeTransactions are created for small batches of writesThe entire set of mutations should be applied within a single -transaction
Data availabilitySurfaces new data to users frequently and quicklyChange sets should be applied atomically, either the effect of -the delta is visible or it is not
Sensitive to record orderNo, records do not have pre-existing lastTxnIds or bucketIds. -Records are likely being written into a single partition (today's date -for example)Yes, all mutated records have existing RecordIdentifiers -and must be grouped by (partitionValues, bucketId) and sorted by -lastTxnId. These record coordinates initially arrive in an order that is -effectively random. -
Impact of a write failureTransaction can be aborted and producer can choose to resubmit -failed records as ordering is not important.Ingest for the respective must be halted and failed records -resubmitted to preserve sequence.
User perception of missing dataData has not arrived yet → "latency?""This data is inconsistent, some records have been updated, but -other related records have not" - consider here the classic transfer -between bank accounts scenario
API end point scopeA given HiveEndPoint instance submits many -transactions to a specific bucket, in a specific partition, of a -specific table -A set ofMutationCoordinators write changes to -unknown set of buckets, of an unknown set of partitions, of specific -tables (can be more than one), within a single transaction. -
- -

Structure

-

The API comprises two main concerns: transaction management, and -the writing of mutation operations to the data set. The two concerns -have a minimal coupling as it is expected that transactions will be -initiated from a single job launcher type processes while the writing of -mutations will be scaled out across any number of worker nodes. In the -context of Hadoop M/R these can be more concretely defined as the Tool -and Map/Reduce task components. However, use of this architecture is not -mandated and in fact both concerns could be handled within a single -simple process depending on the requirements.

- -

Note that a suitably configured Hive instance is required to -operate this system even if you do not intend to access the data from -within Hive. Internally, transactions are managed by the Hive MetaStore. -Mutations are performed to HDFS via ORC APIs that bypass the MetaStore. -Additionally you may wish to configure your MetaStore instance to -perform periodic data compactions.

- -

-Note on packaging: The APIs are defined in the org.apache.hive.hcatalog.streaming.mutate -Java package and included as the hive-hcatalog-streaming jar. -

- -

Data requirements

-

-Generally speaking, to apply a mutation to a record one must have some -unique key that identifies the record. However, primary keys are not a -construct provided by Hive. Internally Hive uses -RecordIdentifiers -stored in a virtual -ROW__ID -column to uniquely identified records within an ACID table. Therefore, -any process that wishes to issue mutations to a table via this API must -have available the corresponding row ids for the target records. What -this means in practice is that the process issuing mutations must first -read in a current snapshot the data and then join the mutations on some -domain specific primary key to obtain the corresponding Hive -ROW__ID -. This is effectively what occurs within Hive's table scan process when -an -UPDATE -or -DELETE -statement is executed. The -AcidInputFormat -provides access to this data via -AcidRecordReader.getRecordIdentifier() -. -

- -

-The implementation of the ACID format places some constraints on the -order in which records are written and it is important that this -ordering is enforced. Additionally, data must be grouped appropriately -to adhere to the constraints imposed be the -OrcRecordUpdater -. Grouping also makes it possible parallelise the writing of mutations -for the purposes of scaling. Finally, to correctly bucket new records -(inserts) there is a slightly unintuitive trick that must be applied. -

- -

All of these data sequencing concerns are the responsibility of -the client process calling the API which is assumed to have first class -grouping and sorting capabilities (Hadoop Map/Reduce etc.) The streaming -API provides nothing more than validators that fail fast when they -encounter groups and records that are out of sequence.

- -

In short, API client processes should prepare data for the mutate -API like so:

-
    -
  • MUST: Order records by ROW__ID.originalTxn, -then ROW__ID.rowId.
  • -
  • MUST: Assign a ROW__ID containing a -computed bucketId to records to be inserted.
  • -
  • SHOULD: Group/partition by table partition value, then ROW__ID.bucketId.
  • -
- -

-The addition of a bucket ids to insert records prior to grouping and -sorting seems unintuitive. However, it is required both to ensure -adequate partitioning of new data and bucket allocation consistent with -that provided by Hive. In a typical ETL the majority of mutation events -are inserts, often targeting a single partition (new data for the -previous day, hour, etc.) If more that one worker is writing said -events, were we to leave the bucket id empty then all inserts would go -to a single worker (e.g: reducer) and the workload could be heavily -skewed. The assignment of a computed bucket allows inserts to be more -usefully distributed across workers. Additionally, when Hive is working -with the data it may expect records to have been bucketed in a way that -is consistent with it's own internal scheme. A convenience type and -method is provided to more easily compute and append bucket ids: -BucketIdResolver -and -BucketIdResolverImpl -. -

- -

Update operations should not attempt to modify values of -partition or bucketing columns. The API does not prevent this and such -attempts could lead to data corruption.

- -

Streaming requirements

-

A few things are currently required to use streaming.

- -

-

    -
  1. Currently, only ORC storage format is supported. So 'stored -as orc' must be specified during table creation. -
  2. -
  3. The hive table must be bucketed, but not sorted. So something -like 'clustered by (colName) into 10 buckets -' must be specified during table creation. -
  4. -
  5. User of the client streaming process must have the necessary -permissions to write to the table or partition and create partitions in -the table.
  6. -
  7. Settings required in hive-site.xml for Metastore: -
      -
    1. hive.txn.manager = -org.apache.hadoop.hive.ql.lockmgr.DbTxnManager
    2. -
    3. hive.support.concurrency = true
    4. -
    5. hive.compactor.initiator.on = true
    6. -
    7. hive.compactor.worker.threads > 0
    8. -
    -
  8. -
-

- -

-Note: Streaming mutations to unpartitioned tables is also -supported. -

- -

Record layout

-

-The structure, layout, and encoding of records is the exclusive concern -of the client ETL mutation process and may be quite different from the -target Hive ACID table. The mutation API requires concrete -implementations of the -MutatorFactory -and -Mutator -classes to extract pertinent data from records and serialize data into -the ACID files. Fortunately base classes are provided ( -AbstractMutator -, -RecordInspectorImpl -) to simplify this effort and usually all that is required is the -specification of a suitable -ObjectInspector -and the provision of the indexes of the -ROW__ID -and bucketed columns within the record structure. Note that all column -indexes in these classes are with respect to your record structure, not -the Hive table structure. -

-

-You will likely also want to use a -BucketIdResolver -to append bucket ids to new records for insertion. Fortunately the core -implementation is provided in -BucketIdResolverImpl -but note that bucket column indexes must be presented in the same order -as they are in the Hive table definition to ensure consistent bucketing. -Note that you cannot move records between buckets and an exception will -be thrown if you attempt to do so. In real terms this mean that you -should not attempt to modify the values in bucket columns with an -UPDATE -. -

- -

Connection and Transaction management

-

-The -MutatorClient -class is used to create and manage transactions in which mutations can -be performed. The scope of a transaction can extend across multiple ACID -tables. When a client connects it communicates with the meta store to -verify and acquire meta data for the target tables. An invocation of -newTransaction -then opens a transaction with the meta store, finalizes a collection of -AcidTables -and returns a new -Transaction -instance. The acid tables are light-weight, serializable objects that -are used by the mutation writing components of the API to target -specific ACID file locations. Usually your -MutatorClient -will be running on some master node and your coordinators on worker -nodes. In this event the -AcidTableSerializer -can be used to encode the tables in a more transportable form, for use -as a -Configuration -property for example. -

-

-As you would expect, a -Transaction -must be initiated with a call to -begin -before any mutations can be applied. This invocation acquires a lock on -the targeted tables using the meta store, and initiates a heartbeat to -prevent transaction timeouts. It is highly recommended that you register -a -LockFailureListener -with the client so that your process can handle any lock or transaction -failures. Typically you may wish to abort the job in the event of such -an error. With the transaction in place you can now start streaming -mutations with one or more -MutatorCoordinator -instances (more on this later), can can finally -commit -or -abort -the transaction when the change set has been applied, which will release -the lock with the meta store client. Finally you should -close -the mutation client to release any held resources. -

-

-The -MutatorClientBuilder -is provided to simplify the construction of clients. -

- -

-WARNING: Hive doesn't currently have a deadlock detector (it is -being worked on as part of HIVE-9675). -This API could potentially deadlock with other stream writers or with -SQL users. -

-

Writing data

- -

-The -MutatorCoordinator -class is used to issue mutations to an ACID table. You will require at -least one instance per table participating in the transaction. The -target of a given instance is defined by the respective -AcidTable -used to construct the coordinator. It is recommended that a -MutatorClientBuilder -is used to simplify the construction process. -

- -

-Mutations can be applied by invoking the respective -insert -, -update -, and -delete -methods on the coordinator. These methods each take as parameters the -target partition of the record and the mutated record. In the case of an -unpartitioned table you should simply pass an empty list as the -partition value. For inserts specifically, only the bucket id will be -extracted from the -RecordIdentifier -, the writeId and rowId will be ignored and replaced by -appropriate values in the -RecordUpdater -. Additionally, in the case of deletes, everything but the -RecordIdentifier -in the record will be ignored and therefore it is often easier to simply -submit the original record. -

- -

-Caution: As mentioned previously, mutations must arrive in -specific order for the resultant table data to be consistent. -Coordinators will verify a naturally ordered sequence of -(writeId, rowId) and will throw an exception if this sequence -is broken. This exception should almost certainly be escalated so that -the transaction is aborted. This, along with the correct ordering of the -data, is the responsibility of the client using the API. -

- -

Dynamic Partition Creation:

-

-It is very likely to be desirable to have new partitions created -automatically (say on a hourly basis). In such cases requiring the Hive -admin to pre-create the necessary partitions may not be reasonable. The -API allows coordinators to create partitions as needed (see: -MutatorClientBuilder.addSinkTable(String, String, boolean) -). Partition creation being an atomic action, multiple coordinators can -race to create the partition, but only one would succeed, so -coordinators clients need not synchronize when creating a partition. The -user of the coordinator process needs to be given write permissions on -the Hive table in order to create partitions. -

- -

Care must be taken when using this option as it requires that the -coordinators maintain a connection with the meta store database. When -coordinator are running in a distributed environment (as is likely the -case) it possible for them to overwhelm the meta store. In such cases it -may be better to disable partition creation and collect a set of -affected partitions as part of your ETL merge process. These can then be -created with a single meta store connection in your client code, once -the cluster side merge process is complete.

-

-Finally, note that when partition creation is disabled the coordinators -must synthesize the partition URI as they cannot retrieve it from the -meta store. This may cause problems if the layout of your partitions in -HDFS does not follow the Hive standard (as implemented in - -org.apache.hadoop.hive.metastore.Warehouse.getPartitionPath(Path, -LinkedHashMap -<String , String>). - -) -

- -

Reading data

- -

-Although this API is concerned with writing changes to data, as -previously stated we'll almost certainly have to read the existing data -first to obtain the relevant -ROW_IDs -. Therefore it is worth noting that reading ACID data in a robust and -consistent manner requires the following: -

    -
  1. Obtaining a valid transaction list from the meta store (ValidTxnList). -
  2. -
  3. Acquiring a lock with the meta store and issuing heartbeats (LockImpl -can help with this). -
  4. -
  5. Configuring the OrcInputFormat and then reading -the data. Make sure that you also pull in the ROW__ID -values. See: AcidRecordReader.getRecordIdentifier. -
  6. -
  7. Releasing the lock.
  8. -
-

- -

Example

-

- -

-

So to recap, the sequence of events required to apply mutations -to a dataset using the API is:

-
    -
  1. Create a MutatorClient to manage a transaction for -the targeted ACID tables. This set of tables should include any -transactional destinations or sources. Don't forget to register a LockFailureListener -so that you can handle transaction failures. -
  2. -
  3. Open a new Transaction with the client. -
  4. -
  5. Get the AcidTables from the client. -
  6. -
  7. Begin the transaction.
  8. -
  9. Create at least one MutatorCoordinator for each -table. The AcidTableSerializer can help you transport the AcidTables -when your workers are in a distributed environment. -
  10. -
  11. Compute your mutation set (this is your ETL merge process).
  12. -
  13. Optionally: collect the set of affected partitions.
  14. -
  15. Append bucket ids to insertion records. A BucketIdResolver -can help here. -
  16. -
  17. Group and sort your data appropriately.
  18. -
  19. Issue mutation events to your coordinators.
  20. -
  21. Close your coordinators.
  22. -
  23. Abort or commit the transaction.
  24. -
  25. Close your mutation client.
  26. -
  27. Optionally: create any affected partitions that do not exist in -the meta store.
  28. -
-

-See -ExampleUseCase -and -TestMutations.testUpdatesAndDeletes() -for some very simple usages. -

- - - - diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java deleted file mode 100644 index 040fce3ec7..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -public class BucketIdException extends WorkerException { - - private static final long serialVersionUID = 1L; - - BucketIdException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java deleted file mode 100644 index 3432baaca4..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -/** Computes and appends bucket ids to records that are due to be inserted. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public interface BucketIdResolver { - - Object attachBucketIdToRecord(Object record); - - /** See: {@link org.apache.hadoop.hive.ql.exec.ReduceSinkOperator#computeBucketNumber(Object, int)}. */ - int computeBucketId(Object record); - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java deleted file mode 100644 index 1d51d8570c..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.util.List; - -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; -import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; - -/** - * Implementation of a {@link BucketIdResolver} that includes the logic required to calculate a bucket id from a record - * that is consistent with Hive's own internal computation scheme. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class BucketIdResolverImpl implements BucketIdResolver { - - private static final long INVALID_TRANSACTION_ID = -1L; - private static final long INVALID_ROW_ID = -1L; - - private final SettableStructObjectInspector structObjectInspector; - private final StructField[] bucketFields; - private final int totalBuckets; - private final StructField recordIdentifierField; - - /** - * Note that all column indexes are with respect to your record structure, not the Hive table structure. Bucket column - * indexes must be presented in the same order as they are in the Hive table definition. - */ - public BucketIdResolverImpl(ObjectInspector objectInspector, int recordIdColumn, int totalBuckets, int[] bucketColumns) { - this.totalBuckets = totalBuckets; - if (!(objectInspector instanceof SettableStructObjectInspector)) { - throw new IllegalArgumentException("Serious problem, expected a StructObjectInspector, " + "but got a " - + objectInspector.getClass().getName()); - } - - if (bucketColumns.length < 1) { - throw new IllegalArgumentException("No bucket column indexes set."); - } - structObjectInspector = (SettableStructObjectInspector) objectInspector; - List structFields = structObjectInspector.getAllStructFieldRefs(); - - recordIdentifierField = structFields.get(recordIdColumn); - - bucketFields = new StructField[bucketColumns.length]; - for (int i = 0; i < bucketColumns.length; i++) { - int bucketColumnsIndex = bucketColumns[i]; - bucketFields[i] = structFields.get(bucketColumnsIndex); - } - } - - @Override - public Object attachBucketIdToRecord(Object record) { - int bucketId = computeBucketId(record); - int bucketProperty = - BucketCodec.V1.encode(new AcidOutputFormat.Options(null).bucket(bucketId)); - RecordIdentifier recordIdentifier = new RecordIdentifier(INVALID_TRANSACTION_ID, bucketProperty, INVALID_ROW_ID); - structObjectInspector.setStructFieldData(record, recordIdentifierField, recordIdentifier); - return record; - } - - @Override - public int computeBucketId(Object record) { - Object[] bucketFieldValues = new Object[bucketFields.length]; - ObjectInspector[] bucketFiledInspectors = new ObjectInspector[bucketFields.length]; - for (int columnIndex = 0; columnIndex < bucketFields.length; columnIndex++) { - bucketFieldValues[columnIndex] = structObjectInspector.getStructFieldData(record, bucketFields[columnIndex]); - bucketFiledInspectors[columnIndex] = bucketFields[columnIndex].getFieldObjectInspector(); - } - return ObjectInspectorUtils.getBucketNumber(bucketFieldValues, bucketFiledInspectors, totalBuckets); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java deleted file mode 100644 index ffa8c3e552..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -public class GroupRevisitedException extends WorkerException { - - private static final long serialVersionUID = 1L; - - GroupRevisitedException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java deleted file mode 100644 index f28b8ffe60..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -/** - * Tracks the (partition, bucket) combinations that have been encountered, checking that a group is not revisited. - * Potentially memory intensive. - */ -class GroupingValidator { - - private final Map> visited; - private final StringBuilder partitionKeyBuilder; - private long groups; - private String lastPartitionKey; - private int lastBucketId = -1; - - GroupingValidator() { - visited = new HashMap>(); - partitionKeyBuilder = new StringBuilder(64); - } - - /** - * Checks that this group is either the same as the last or is a new group. - */ - boolean isInSequence(List partitionValues, int bucketId) { - String partitionKey = getPartitionKey(partitionValues); - if (Objects.equals(lastPartitionKey, partitionKey) && lastBucketId == bucketId) { - return true; - } - lastPartitionKey = partitionKey; - lastBucketId = bucketId; - - Set bucketIdSet = visited.get(partitionKey); - if (bucketIdSet == null) { - // If the bucket id set component of this data structure proves to be too large there is the - // option of moving it to Trove or HPPC in an effort to reduce size. - bucketIdSet = new HashSet<>(); - visited.put(partitionKey, bucketIdSet); - } - - boolean newGroup = bucketIdSet.add(bucketId); - if (newGroup) { - groups++; - } - return newGroup; - } - - private String getPartitionKey(List partitionValues) { - partitionKeyBuilder.setLength(0); - boolean first = true; - for (String element : partitionValues) { - if (first) { - first = false; - } else { - partitionKeyBuilder.append('/'); - } - partitionKeyBuilder.append(element); - } - String partitionKey = partitionKeyBuilder.toString(); - return partitionKey; - } - - @Override - public String toString() { - return "GroupingValidator [groups=" + groups + ",lastPartitionKey=" + lastPartitionKey + ",lastBucketId=" - + lastBucketId + "]"; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java deleted file mode 100644 index fb88f2d081..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.thrift.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A {@link PartitionHelper} implementation that uses the {@link IMetaStoreClient meta store} to both create partitions - * and obtain information concerning partitions. Exercise care when using this from within workers that are running in a - * cluster as it may overwhelm the meta store database instance. As an alternative, consider using the - * {@link WarehousePartitionHelper}, collecting the affected partitions as an output of your merge job, and then - * retrospectively adding partitions in your client. - */ -class MetaStorePartitionHelper implements PartitionHelper { - - private static final Logger LOG = LoggerFactory.getLogger(MetaStorePartitionHelper.class); - - private final IMetaStoreClient metaStoreClient; - private final String databaseName; - private final String tableName; - private final Path tablePath; - - MetaStorePartitionHelper(IMetaStoreClient metaStoreClient, String databaseName, String tableName, Path tablePath) { - this.metaStoreClient = metaStoreClient; - this.tablePath = tablePath; - this.databaseName = databaseName; - this.tableName = tableName; - } - - /** Returns the expected {@link Path} for a given partition value. */ - @Override - public Path getPathForPartition(List newPartitionValues) throws WorkerException { - if (newPartitionValues.isEmpty()) { - LOG.debug("Using path {} for unpartitioned table {}.{}", tablePath, databaseName, tableName); - return tablePath; - } else { - try { - String location = metaStoreClient - .getPartition(databaseName, tableName, newPartitionValues) - .getSd() - .getLocation(); - LOG.debug("Found path {} for partition {}", location, newPartitionValues); - return new Path(location); - } catch (NoSuchObjectException e) { - throw new WorkerException("Table not found '" + databaseName + "." + tableName + "'.", e); - } catch (TException e) { - throw new WorkerException("Failed to get path for partitions '" + newPartitionValues + "' on table '" - + databaseName + "." + tableName + "' with meta store: " + metaStoreClient, e); - } - } - } - - /** Creates the specified partition if it does not already exist. Does nothing if the table is unpartitioned. */ - @Override - public void createPartitionIfNotExists(List newPartitionValues) throws WorkerException { - if (newPartitionValues.isEmpty()) { - return; - } - - try { - LOG.debug("Attempting to create partition (if not exists) {}.{}:{}", databaseName, tableName, newPartitionValues); - Table table = metaStoreClient.getTable(databaseName, tableName); - - Partition partition = new Partition(); - partition.setDbName(table.getDbName()); - partition.setTableName(table.getTableName()); - StorageDescriptor partitionSd = new StorageDescriptor(table.getSd()); - partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR - + Warehouse.makePartName(table.getPartitionKeys(), newPartitionValues)); - partition.setSd(partitionSd); - partition.setValues(newPartitionValues); - - metaStoreClient.add_partition(partition); - } catch (AlreadyExistsException e) { - LOG.debug("Partition already exisits: {}.{}:{}", databaseName, tableName, newPartitionValues); - } catch (NoSuchObjectException e) { - LOG.error("Failed to create partition : " + newPartitionValues, e); - throw new PartitionCreationException("Table not found '" + databaseName + "." + tableName + "'.", e); - } catch (TException e) { - LOG.error("Failed to create partition : " + newPartitionValues, e); - throw new PartitionCreationException("Failed to create partition '" + newPartitionValues + "' on table '" - + databaseName + "." + tableName + "'", e); - } - } - - @Override - public void close() throws IOException { - metaStoreClient.close(); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java deleted file mode 100644 index e6f968eff5..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.Closeable; -import java.io.Flushable; -import java.io.IOException; - -/** - * Interface for submitting mutation events to a given partition and bucket in an ACID table. Requires records to arrive - * in the order defined by the {@link SequenceValidator}. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public interface Mutator extends Closeable, Flushable { - - void insert(Object record) throws IOException; - - void update(Object record) throws IOException; - - void delete(Object record) throws IOException; - - void flush() throws IOException; - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java deleted file mode 100644 index a90d5d3187..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java +++ /dev/null @@ -1,300 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.Closeable; -import java.io.Flushable; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.ql.io.RecordUpdater; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Orchestrates the application of an ordered sequence of mutation events to a given ACID table. Events must be grouped - * by partition, then bucket and ordered by origTxnId, then rowId. Ordering is enforced by the {@link SequenceValidator} - * and grouping is by the {@link GroupingValidator}. An acid delta file is created for each combination partition, and - * bucket id (a single write id is implied). Once a delta file has been closed it cannot be reopened. Therefore - * care is needed as to group the data correctly otherwise failures will occur if a delta belonging to group has been - * previously closed. The {@link MutatorCoordinator} will seamlessly handle transitions between groups, creating and - * closing {@link Mutator Mutators} as needed to write to the appropriate partition and bucket. New partitions will be - * created in the meta store if {@link AcidTable#createPartitions()} is set. - *

- * {@link #insert(List, Object) Insert} events must be artificially assigned appropriate bucket ids in the preceding - * grouping phase so that they are grouped correctly. Note that any write id or row id assigned to the - * {@link RecordIdentifier RecordIdentifier} of such events will be ignored by both the coordinator and the underlying - * {@link RecordUpdater}. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class MutatorCoordinator implements Closeable, Flushable { - - private static final Logger LOG = LoggerFactory.getLogger(MutatorCoordinator.class); - - private final MutatorFactory mutatorFactory; - private final GroupingValidator groupingValidator; - private final SequenceValidator sequenceValidator; - private final AcidTable table; - private final RecordInspector recordInspector; - private final PartitionHelper partitionHelper; - private final AcidOutputFormat outputFormat; - private final BucketIdResolver bucketIdResolver; - private final HiveConf configuration; - private final boolean deleteDeltaIfExists; - - private int bucketId; - private List partitionValues; - private Path partitionPath; - private Mutator mutator; - - MutatorCoordinator(HiveConf configuration, MutatorFactory mutatorFactory, PartitionHelper partitionHelper, - AcidTable table, boolean deleteDeltaIfExists) throws WorkerException { - this(configuration, mutatorFactory, partitionHelper, new GroupingValidator(), new SequenceValidator(), table, - deleteDeltaIfExists); - } - - /** Visible for testing only. */ - MutatorCoordinator(HiveConf configuration, MutatorFactory mutatorFactory, PartitionHelper partitionHelper, - GroupingValidator groupingValidator, SequenceValidator sequenceValidator, AcidTable table, - boolean deleteDeltaIfExists) throws WorkerException { - this.configuration = configuration; - this.mutatorFactory = mutatorFactory; - this.partitionHelper = partitionHelper; - this.groupingValidator = groupingValidator; - this.sequenceValidator = sequenceValidator; - this.table = table; - this.deleteDeltaIfExists = deleteDeltaIfExists; - this.recordInspector = this.mutatorFactory.newRecordInspector(); - bucketIdResolver = this.mutatorFactory.newBucketIdResolver(table.getTotalBuckets()); - - bucketId = -1; - outputFormat = createOutputFormat(table.getOutputFormatName(), configuration); - } - - /** - * We expect records grouped by (partitionValues,bucketId) and ordered by (origWriteId,rowId). - * - * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed - * using the values in the record's bucketed columns. - * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origWriteId, rowId) - * sequence. - * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already - * been closed. - * @throws PartitionCreationException Could not create a new partition in the meta store. - * @throws WorkerException - */ - public void insert(List partitionValues, Object record) throws WorkerException { - reconfigureState(OperationType.INSERT, partitionValues, record); - try { - mutator.insert(record); - LOG.debug("Inserted into partition={}, record={}", partitionValues, record); - } catch (IOException e) { - throw new WorkerException("Failed to insert record '" + record + " using mutator '" + mutator + "'.", e); - } - } - - /** - * We expect records grouped by (partitionValues,bucketId) and ordered by (origWriteId,rowId). - * - * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed - * using the values in the record's bucketed columns. - * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origWriteId, rowId) - * sequence. - * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already - * been closed. - * @throws PartitionCreationException Could not create a new partition in the meta store. - * @throws WorkerException - */ - public void update(List partitionValues, Object record) throws WorkerException { - reconfigureState(OperationType.UPDATE, partitionValues, record); - try { - mutator.update(record); - LOG.debug("Updated in partition={}, record={}", partitionValues, record); - } catch (IOException e) { - throw new WorkerException("Failed to update record '" + record + " using mutator '" + mutator + "'.", e); - } - } - - /** - * We expect records grouped by (partitionValues,bucketId) and ordered by (origWriteId,rowId). - * - * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed - * using the values in the record's bucketed columns. - * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origWriteId, rowId) - * sequence. - * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already - * been closed. - * @throws PartitionCreationException Could not create a new partition in the meta store. - * @throws WorkerException - */ - public void delete(List partitionValues, Object record) throws WorkerException { - reconfigureState(OperationType.DELETE, partitionValues, record); - try { - mutator.delete(record); - LOG.debug("Deleted from partition={}, record={}", partitionValues, record); - } catch (IOException e) { - throw new WorkerException("Failed to delete record '" + record + " using mutator '" + mutator + "'.", e); - } - } - - @Override - public void close() throws IOException { - try { - if (mutator != null) { - mutator.close(); - } - } finally { - partitionHelper.close(); - } - } - - @Override - public void flush() throws IOException { - if (mutator != null) { - mutator.flush(); - } - } - - private void reconfigureState(OperationType operationType, List newPartitionValues, Object record) - throws WorkerException { - RecordIdentifier newRecordIdentifier = extractRecordIdentifier(operationType, newPartitionValues, record); - int newBucketId = newRecordIdentifier.getBucketProperty(); - - if (newPartitionValues == null) { - newPartitionValues = Collections.emptyList(); - } - - try { - if (partitionHasChanged(newPartitionValues)) { - if (table.createPartitions() && operationType == OperationType.INSERT) { - partitionHelper.createPartitionIfNotExists(newPartitionValues); - } - Path newPartitionPath = partitionHelper.getPathForPartition(newPartitionValues); - resetMutator(newBucketId, newPartitionValues, newPartitionPath); - } else if (bucketIdHasChanged(newBucketId)) { - resetMutator(newBucketId, partitionValues, partitionPath); - } else { - validateRecordSequence(operationType, newRecordIdentifier); - } - } catch (IOException e) { - throw new WorkerException("Failed to reset mutator when performing " + operationType + " of record: " + record, e); - } - } - - private RecordIdentifier extractRecordIdentifier(OperationType operationType, List newPartitionValues, - Object record) throws BucketIdException { - RecordIdentifier recordIdentifier = recordInspector.extractRecordIdentifier(record); - int bucketIdFromRecord = BucketCodec.determineVersion( - recordIdentifier.getBucketProperty()).decodeWriterId(recordIdentifier.getBucketProperty()); - int computedBucketId = bucketIdResolver.computeBucketId(record); - if (operationType != OperationType.DELETE && bucketIdFromRecord != computedBucketId) { - throw new BucketIdException("RecordIdentifier.bucketId != computed bucketId (" + computedBucketId - + ") for record " + recordIdentifier + " in partition " + newPartitionValues + "."); - } - return recordIdentifier; - } - - private void resetMutator(int newBucketId, List newPartitionValues, Path newPartitionPath) - throws IOException, GroupRevisitedException { - if (mutator != null) { - mutator.close(); - } - validateGrouping(newPartitionValues, newBucketId); - sequenceValidator.reset(); - if (deleteDeltaIfExists) { - // TODO: Should this be the concern of the mutator? - deleteDeltaIfExists(newPartitionPath, table.getWriteId(), newBucketId); - } - mutator = mutatorFactory.newMutator(outputFormat, table.getWriteId(), newPartitionPath, newBucketId); - bucketId = newBucketId; - partitionValues = newPartitionValues; - partitionPath = newPartitionPath; - LOG.debug("Reset mutator: bucketId={}, partition={}, partitionPath={}", bucketId, partitionValues, partitionPath); - } - - private boolean partitionHasChanged(List newPartitionValues) { - boolean partitionHasChanged = !Objects.equals(this.partitionValues, newPartitionValues); - if (partitionHasChanged) { - LOG.debug("Partition changed from={}, to={}", this.partitionValues, newPartitionValues); - } - return partitionHasChanged; - } - - private boolean bucketIdHasChanged(int newBucketId) { - boolean bucketIdHasChanged = this.bucketId != newBucketId; - if (bucketIdHasChanged) { - LOG.debug("Bucket ID changed from={}, to={}", this.bucketId, newBucketId); - } - return bucketIdHasChanged; - } - - private void validateGrouping(List newPartitionValues, int newBucketId) throws GroupRevisitedException { - if (!groupingValidator.isInSequence(newPartitionValues, bucketId)) { - throw new GroupRevisitedException("Group out of sequence: state=" + groupingValidator + ", partition=" - + newPartitionValues + ", bucketId=" + newBucketId); - } - } - - private void validateRecordSequence(OperationType operationType, RecordIdentifier newRecordIdentifier) - throws RecordSequenceException { - boolean identiferOutOfSequence = operationType != OperationType.INSERT - && !sequenceValidator.isInSequence(newRecordIdentifier); - if (identiferOutOfSequence) { - throw new RecordSequenceException("Records not in sequence: state=" + sequenceValidator + ", recordIdentifier=" - + newRecordIdentifier); - } - } - - @SuppressWarnings("unchecked") - private AcidOutputFormat createOutputFormat(String outputFormatName, HiveConf configuration) - throws WorkerException { - try { - return (AcidOutputFormat) ReflectionUtils.newInstance(JavaUtils.loadClass(outputFormatName), configuration); - } catch (ClassNotFoundException e) { - throw new WorkerException("Could not locate class for '" + outputFormatName + "'.", e); - } - } - - /* A delta may be present from a previous failed task attempt. */ - private void deleteDeltaIfExists(Path partitionPath, long writeId, int bucketId) throws IOException { - Path deltaPath = AcidUtils.createFilename(partitionPath, - new AcidOutputFormat.Options(configuration) - .bucket(bucketId) - .minimumWriteId(writeId) - .maximumWriteId(writeId)); - FileSystem fileSystem = deltaPath.getFileSystem(configuration); - if (fileSystem.exists(deltaPath)) { - LOG.info("Deleting existing delta path: {}", deltaPath); - fileSystem.delete(deltaPath, false); - } - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java deleted file mode 100644 index 698ba7cb35..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hive.hcatalog.common.HCatUtil; -import org.apache.hive.hcatalog.streaming.mutate.HiveConfFactory; -import org.apache.hive.hcatalog.streaming.mutate.UgiMetaStoreClientFactory; -import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable; - -/** Convenience class for building {@link MutatorCoordinator} instances. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class MutatorCoordinatorBuilder { - - private HiveConf configuration; - private MutatorFactory mutatorFactory; - private UserGroupInformation authenticatedUser; - private String metaStoreUri; - private AcidTable table; - private boolean deleteDeltaIfExists; - - public MutatorCoordinatorBuilder configuration(HiveConf configuration) { - this.configuration = configuration; - return this; - } - - public MutatorCoordinatorBuilder authenticatedUser(UserGroupInformation authenticatedUser) { - this.authenticatedUser = authenticatedUser; - return this; - } - - public MutatorCoordinatorBuilder metaStoreUri(String metaStoreUri) { - this.metaStoreUri = metaStoreUri; - return this; - } - - /** Set the destination ACID table for this client. */ - public MutatorCoordinatorBuilder table(AcidTable table) { - this.table = table; - return this; - } - - /** - * If the delta file already exists, delete it. THis is useful in a MapReduce setting where a number of task retries - * will attempt to write the same delta file. - */ - public MutatorCoordinatorBuilder deleteDeltaIfExists() { - this.deleteDeltaIfExists = true; - return this; - } - - public MutatorCoordinatorBuilder mutatorFactory(MutatorFactory mutatorFactory) { - this.mutatorFactory = mutatorFactory; - return this; - } - - public MutatorCoordinator build() throws WorkerException, MetaException { - configuration = HiveConfFactory.newInstance(configuration, this.getClass(), metaStoreUri); - - PartitionHelper partitionHelper; - if (table.createPartitions()) { - partitionHelper = newMetaStorePartitionHelper(); - } else { - partitionHelper = newWarehousePartitionHelper(); - } - - return new MutatorCoordinator(configuration, mutatorFactory, partitionHelper, table, deleteDeltaIfExists); - } - - private PartitionHelper newWarehousePartitionHelper() throws MetaException, WorkerException { - String location = table.getTable().getSd().getLocation(); - Path tablePath = new Path(location); - List partitionFields = table.getTable().getPartitionKeys(); - List partitionColumns = new ArrayList<>(partitionFields.size()); - for (FieldSchema field : partitionFields) { - partitionColumns.add(field.getName()); - } - return new WarehousePartitionHelper(configuration, tablePath, partitionColumns); - } - - private PartitionHelper newMetaStorePartitionHelper() throws MetaException, WorkerException { - String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName(); - boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials(); - try { - IMetaStoreClient metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser, - user, secureMode).newInstance(HCatUtil.getHiveMetastoreClient(configuration)); - String tableLocation = table.getTable().getSd().getLocation(); - Path tablePath = new Path(tableLocation); - return new MetaStorePartitionHelper(metaStoreClient, table.getDatabaseName(), table.getTableName(), tablePath); - } catch (IOException e) { - throw new WorkerException("Could not create meta store client.", e); - } - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java deleted file mode 100644 index d3d3210ad4..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.IOException; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; - -/** - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public interface MutatorFactory { - - Mutator newMutator(AcidOutputFormat outputFormat, long writeId, Path partitionPath, int bucketId) - throws IOException; - - RecordInspector newRecordInspector(); - - BucketIdResolver newBucketIdResolver(int totalBuckets); - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java deleted file mode 100644 index 1e0cb727ec..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.ql.io.RecordUpdater; -import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; - -/** Base {@link Mutator} implementation. Creates a suitable {@link RecordUpdater} and delegates mutation events. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class MutatorImpl implements Mutator { - - private final long writeId; - private final Path partitionPath; - private final int bucketProperty; - private final Configuration configuration; - private final int recordIdColumn; - private final ObjectInspector objectInspector; - private RecordUpdater updater; - - /** - * @param bucketProperty - from existing {@link RecordIdentifier#getBucketProperty()} - * @throws IOException - */ - public MutatorImpl(Configuration configuration, int recordIdColumn, ObjectInspector objectInspector, - AcidOutputFormat outputFormat, long writeId, Path partitionPath, int bucketProperty) throws IOException { - this.configuration = configuration; - this.recordIdColumn = recordIdColumn; - this.objectInspector = objectInspector; - this.writeId = writeId; - this.partitionPath = partitionPath; - this.bucketProperty = bucketProperty; - - updater = createRecordUpdater(outputFormat); - } - - @Override - public void insert(Object record) throws IOException { - updater.insert(writeId, record); - } - - @Override - public void update(Object record) throws IOException { - updater.update(writeId, record); - } - - @Override - public void delete(Object record) throws IOException { - updater.delete(writeId, record); - } - - /** - * This implementation does intentionally nothing at this time. We only use a single transaction and - * {@link OrcRecordUpdater#flush()} will purposefully throw and exception in this instance. We keep this here in the - * event that we support multiple transactions and to make it clear that the omission of an invocation of - * {@link OrcRecordUpdater#flush()} was not a mistake. - */ - @Override - public void flush() throws IOException { - // Intentionally do nothing - } - - @Override - public void close() throws IOException { - updater.close(false); - updater = null; - } - - @Override - public String toString() { - return "ObjectInspectorMutator [writeId=" + writeId + ", partitionPath=" + partitionPath - + ", bucketId=" + bucketProperty + "]"; - } - - protected RecordUpdater createRecordUpdater(AcidOutputFormat outputFormat) throws IOException { - int bucketId = BucketCodec - .determineVersion(bucketProperty).decodeWriterId(bucketProperty); - return outputFormat.getRecordUpdater( - partitionPath, - new AcidOutputFormat.Options(configuration) - .inspector(objectInspector) - .bucket(bucketId) - .minimumWriteId(writeId) - .maximumWriteId(writeId) - .recordIdColumn(recordIdColumn) - .finalDestination(partitionPath) - .statementId(-1)); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java deleted file mode 100644 index 3dc2886b5a..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -enum OperationType { - INSERT, - UPDATE, - DELETE; -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java deleted file mode 100644 index ed0c9890b5..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -public class PartitionCreationException extends WorkerException { - - private static final long serialVersionUID = 1L; - - PartitionCreationException(String message, Throwable cause) { - super(message, cause); - } - - PartitionCreationException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java deleted file mode 100644 index d064b0c701..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.Closeable; -import java.util.List; - -import org.apache.hadoop.fs.Path; - -/** Implementations are responsible for creating and obtaining path information about partitions. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -interface PartitionHelper extends Closeable { - - /** Return the location of the partition described by the provided values. */ - Path getPathForPartition(List newPartitionValues) throws WorkerException; - - /** Create the partition described by the provided values if it does not exist already. */ - void createPartitionIfNotExists(List newPartitionValues) throws WorkerException; - -} \ No newline at end of file diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java deleted file mode 100644 index 5d1f17513b..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import org.apache.hadoop.hive.ql.io.RecordIdentifier; - -/** Provide a means to extract {@link RecordIdentifier} from record objects. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public interface RecordInspector { - - /** Get the {@link RecordIdentifier} from the record - to be used for updates and deletes only. */ - RecordIdentifier extractRecordIdentifier(Object record); - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java deleted file mode 100644 index 37329c33c7..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.util.List; - -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; - -/** - * Standard {@link RecordInspector} implementation that uses the supplied {@link ObjectInspector} and - * {@link AcidOutputFormat.Options#recordIdColumn(int) record id column} to extract {@link RecordIdentifier - * RecordIdentifiers}, and calculate bucket ids from records. - * @deprecated as of Hive 3.0.0 - */ -@Deprecated -public class RecordInspectorImpl implements RecordInspector { - - private final StructObjectInspector structObjectInspector; - private final StructField recordIdentifierField; - - /** - * Note that all column indexes are with respect to your record structure, not the Hive table structure. - */ - public RecordInspectorImpl(ObjectInspector objectInspector, int recordIdColumn) { - if (!(objectInspector instanceof StructObjectInspector)) { - throw new IllegalArgumentException("Serious problem, expected a StructObjectInspector, " + "but got a " - + objectInspector.getClass().getName()); - } - - structObjectInspector = (StructObjectInspector) objectInspector; - List structFields = structObjectInspector.getAllStructFieldRefs(); - recordIdentifierField = structFields.get(recordIdColumn); - } - - public RecordIdentifier extractRecordIdentifier(Object record) { - return (RecordIdentifier) structObjectInspector.getStructFieldData(record, recordIdentifierField); - } - - @Override - public String toString() { - return "RecordInspectorImpl [structObjectInspector=" + structObjectInspector + ", recordIdentifierField=" - + recordIdentifierField + "]"; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java deleted file mode 100644 index 0d3b471488..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -public class RecordSequenceException extends WorkerException { - - private static final long serialVersionUID = 1L; - - RecordSequenceException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java deleted file mode 100644 index 320b987b84..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Verifies that the sequence of {@link RecordIdentifier RecordIdentifiers} are in a valid order for insertion into an - * ACID delta file in a given partition and bucket. - */ -class SequenceValidator { - - private static final Logger LOG = LoggerFactory.getLogger(SequenceValidator.class); - - private Long lastWriteId; - private Long lastRowId; - - SequenceValidator() { - } - - boolean isInSequence(RecordIdentifier recordIdentifier) { - if (lastWriteId != null && recordIdentifier.getWriteId() < lastWriteId) { - LOG.debug("Non-sequential write ID. Expected >{}, recordIdentifier={}", lastWriteId, recordIdentifier); - return false; - } else if (lastWriteId != null && recordIdentifier.getWriteId() == lastWriteId && lastRowId != null - && recordIdentifier.getRowId() <= lastRowId) { - LOG.debug("Non-sequential row ID. Expected >{}, recordIdentifier={}", lastRowId, recordIdentifier); - return false; - } - lastWriteId = recordIdentifier.getWriteId(); - lastRowId = recordIdentifier.getRowId(); - return true; - } - - /** - * Validator must be reset for each new partition and or bucket. - */ - void reset() { - lastWriteId = null; - lastRowId = null; - LOG.debug("reset"); - } - - @Override - public String toString() { - return "SequenceValidator [lastWriteId=" + lastWriteId + ", lastRowId=" + lastRowId + "]"; - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java deleted file mode 100644 index ace329ab04..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import java.io.IOException; -import java.util.LinkedHashMap; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.MetaException; - -/** - * A {@link PartitionHelper} implementation that uses the {@link Warehouse} class to obtain partition path information. - * As this does not require a connection to the meta store database it is safe to use in workers that are distributed on - * a cluster. However, it does not support the creation of new partitions so you will need to provide a mechanism to - * collect affected partitions in your merge job and create them from your client. - */ -class WarehousePartitionHelper implements PartitionHelper { - - private final Warehouse warehouse; - private final Path tablePath; - private final LinkedHashMap partitions; - private final List partitionColumns; - - WarehousePartitionHelper(Configuration configuration, Path tablePath, List partitionColumns) - throws MetaException { - this.tablePath = tablePath; - this.partitionColumns = partitionColumns; - this.partitions = new LinkedHashMap<>(partitionColumns.size()); - for (String partitionColumn : partitionColumns) { - partitions.put(partitionColumn, null); - } - warehouse = new Warehouse(configuration); - } - - @Override - public Path getPathForPartition(List partitionValues) throws WorkerException { - if (partitionValues.size() != partitionColumns.size()) { - throw new IllegalArgumentException("Incorrect number of partition values. columns=" + partitionColumns - + ",values=" + partitionValues); - } - if (partitionColumns.isEmpty()) { - return tablePath; - } - for (int columnIndex = 0; columnIndex < partitionValues.size(); columnIndex++) { - String partitionColumn = partitionColumns.get(columnIndex); - String partitionValue = partitionValues.get(columnIndex); - partitions.put(partitionColumn, partitionValue); - } - try { - return warehouse.getPartitionPath(tablePath, partitions); - } catch (MetaException e) { - throw new WorkerException("Unable to determine partition path. tablePath=" + tablePath + ",partition=" - + partitionValues, e); - } - } - - /** Throws {@link UnsupportedOperationException}. */ - @Override - public void createPartitionIfNotExists(List newPartitionValues) throws WorkerException { - throw new UnsupportedOperationException("You require a connection to the meta store to do this."); - } - - @Override - public void close() throws IOException { - // Nothing to close here. - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java deleted file mode 100644 index 9eb67427d6..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -public class WorkerException extends Exception { - - private static final long serialVersionUID = 1L; - - WorkerException(String message, Throwable cause) { - super(message, cause); - } - - WorkerException(String message) { - super(message); - } - -} diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package.html b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package.html deleted file mode 100644 index a879b974ae..0000000000 --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package.html +++ /dev/null @@ -1,181 +0,0 @@ - - - - - - - - -HCatalog Streaming API - - - - -

HCatalog Streaming API -- high level description

- -NOTE: The Streaming API feature is provided as a technology -preview. The API may undergo incompatible changes in upcoming -releases. - -

-Traditionally adding new data into hive requires gathering a large -amount of data onto HDFS and then periodically adding a new -partition. This is essentially a batch insertion. Insertion of -new data into an existing partition or table is not done in a way that -gives consistent results to readers. Hive Streaming API allows data to -be pumped continuously into Hive. The incoming data can be -continuously committed in small batches (of records) into a Hive -partition. Once data is committed it becomes immediately visible to -all Hive queries initiated subsequently.

- -

-This API is intended for streaming clients such as NiFi, Flume and Storm, -which continuously generate data. Streaming support is built on top of -ACID based insert/update support in Hive.

- -

-The classes and interfaces part of the Hive streaming API are broadly -categorized into two. The first set provides support for connection -and transaction management while the second set provides I/O -support. Transactions are managed by the Hive MetaStore. Writes are -performed to HDFS via Hive wrapper APIs that bypass MetaStore.

- -

-Note on packaging: The APIs are defined in the -org.apache.hive.hcatalog.streaming Java package and included as -the hive-hcatalog-streaming jar.

- -

STREAMING REQUIREMENTS

- -

-A few things are currently required to use streaming. -

- -

-

    -
  1. Currently, only ORC storage format is supported. So - 'stored as orc' must be specified during table creation.
  2. -
  3. The hive table may be bucketed but must not be sorted.
  4. -
  5. User of the client streaming process must have the necessary - permissions to write to the table or partition and create partitions in - the table.
  6. -
  7. Currently, when issuing queries on streaming tables, query client must set -
      -
    1. hive.input.format = - org.apache.hadoop.hive.ql.io.HiveInputFormat
    2. -
  8. - The above client settings are a temporary requirement and the intention is to - drop the need for them in the near future. -
  9. Settings required in hive-site.xml for Metastore: -
      -
    1. hive.txn.manager = - org.apache.hadoop.hive.ql.lockmgr.DbTxnManager
    2. -
    3. hive.support.concurrency = true
    4. -
    5. hive.compactor.initiator.on = true
    6. -
    7. hive.compactor.worker.threads > 0
    8. -
  10. -

- -

-Note: Streaming to unpartitioned tables is also -supported.

- -

Transaction and Connection management

- -

-The class HiveEndPoint is a Hive end -point to connect to. An endpoint is either a Hive table or -partition. An endpoint is cheap to create and does not internally hold -on to any network connections. Invoking the newConnection method on -it creates a new connection to the Hive MetaStore for streaming -purposes. It returns a -StreamingConnection -object. Multiple connections can be established on the same -endpoint. StreamingConnection can then be used to initiate new -transactions for performing I/O.

- -

Dynamic Partition Creation:

It is very likely that a setup in -which data is being streamed continuously (e.g. Flume), it is -desirable to have new partitions created automatically (say on a -hourly basis). In such cases requiring the Hive admin to pre-create -the necessary partitions may not be reasonable. Consequently the -streaming API allows streaming clients to create partitions as -needed. HiveEndPoind.newConnection() accepts a argument to -indicate if the partition should be auto created. Partition creation -being an atomic action, multiple clients can race to create the -partition, but only one would succeed, so streaming clients need not -synchronize when creating a partition. The user of the client process -needs to be given write permissions on the Hive table in order to -create partitions. - -

Batching Transactions:

Transactions are implemented slightly -differently than traditional database systems. Multiple transactions -are grouped into a Transaction Batch and each transaction has -an id. Data from each transaction batch gets a single file on HDFS, -which eventually gets compacted with other files into a larger file -automatically for efficiency. - -

Basic Steps:

After connection is established, a streaming -client first requests for a new batch of transactions. In response it -receives a set of transaction ids that are part of the transaction -batch. Subsequently the client proceeds to consume one transaction at -a time by initiating new transactions. Client will write() one or more -records per transactions and either commit or abort the current -transaction before switching to the next one. Each -TransactionBatch.write() invocation automatically associates -the I/O attempt with the current transaction id. The user of the -streaming client needs to have write permissions to the partition or -table.

- -

-Concurrency Note: I/O can be performed on multiple -TransactionBatchs concurrently. However the transactions within a -transaction batch much be consumed sequentially.

- -

Writing Data

- -

-These classes and interfaces provide support for writing the data to -Hive within a transaction. -RecordWriter is the interface -implemented by all writers. A writer is responsible for taking a -record in the form of a byte[] containing data in a known -format (e.g. CSV) and writing it out in the format supported by Hive -streaming. A RecordWriter may reorder or drop fields from the incoming -record if necessary to map them to the corresponding columns in the -Hive Table. A streaming client will instantiate an appropriate -RecordWriter type and pass it to -StreamingConnection.fetchTransactionBatch(). The streaming client -does not directly interact with the RecordWriter therafter, but -relies on the TransactionBatch to do so.

- -

-Currently, out of the box, the streaming API provides two -implementations of the RecordWriter interface. One handles delimited -input data (such as CSV, tab separated, etc. and the other for JSON -(strict syntax). Support for other input formats can be provided by -additional implementations of the RecordWriter interface. -

- -

Performance, Concurrency, Etc.

-

- Each StreamingConnection is writing data at the rate the underlying - FileSystem can accept it. If that is not sufficient, multiple StreamingConnection objects can - be created concurrently. -

-

- Each StreamingConnection can have at most 1 outstanding TransactionBatch and each TransactionBatch - may have at most 2 threads operaing on it. - See TransactionBatch -

- - - diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/StreamingIntegrationTester.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/StreamingIntegrationTester.java deleted file mode 100644 index af252aa0ab..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/StreamingIntegrationTester.java +++ /dev/null @@ -1,347 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming; - -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.commons.cli.ParseException; -import org.apache.commons.cli.Parser; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.LogUtils; -import org.apache.hadoop.util.StringUtils; - -import java.util.Arrays; -import java.util.Random; - -/** - * A stand alone utility to write data into the streaming ingest interface. - */ -public class StreamingIntegrationTester { - - static final private Logger LOG = LoggerFactory.getLogger(StreamingIntegrationTester.class.getName()); - - public static void main(String[] args) { - - try { - LogUtils.initHiveLog4j(); - } catch (LogUtils.LogInitializationException e) { - System.err.println("Unable to initialize log4j " + StringUtils.stringifyException(e)); - System.exit(-1); - } - - Options options = new Options(); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("abort-pct") - .withDescription("Percentage of transactions to abort, defaults to 5") - .withLongOpt("abortpct") - .create('a')); - - options.addOption(OptionBuilder - .hasArgs() - .withArgName("column-names") - .withDescription("column names of table to write to") - .withLongOpt("columns") - .withValueSeparator(',') - .isRequired() - .create('c')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("database") - .withDescription("Database of table to write to") - .withLongOpt("database") - .isRequired() - .create('d')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("frequency") - .withDescription("How often to commit a transaction, in seconds, defaults to 1") - .withLongOpt("frequency") - .create('f')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("iterations") - .withDescription("Number of batches to write, defaults to 10") - .withLongOpt("num-batches") - .create('i')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("metastore-uri") - .withDescription("URI of Hive metastore") - .withLongOpt("metastore-uri") - .isRequired() - .create('m')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("num_transactions") - .withDescription("Number of transactions per batch, defaults to 100") - .withLongOpt("num-txns") - .create('n')); - - options.addOption(OptionBuilder - .hasArgs() - .withArgName("partition-values") - .withDescription("partition values, must be provided in order of partition columns, " + - "if not provided table is assumed to not be partitioned") - .withLongOpt("partition") - .withValueSeparator(',') - .create('p')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("records-per-transaction") - .withDescription("records to write in each transaction, defaults to 100") - .withLongOpt("records-per-txn") - .withValueSeparator(',') - .create('r')); - - options.addOption(OptionBuilder - .hasArgs() - .withArgName("column-types") - .withDescription("column types, valid values are string, int, float, decimal, date, " + - "datetime") - .withLongOpt("schema") - .withValueSeparator(',') - .isRequired() - .create('s')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("table") - .withDescription("Table to write to") - .withLongOpt("table") - .isRequired() - .create('t')); - - options.addOption(OptionBuilder - .hasArg() - .withArgName("num-writers") - .withDescription("Number of writers to create, defaults to 2") - .withLongOpt("writers") - .create('w')); - - options.addOption(OptionBuilder - .hasArg(false) - .withArgName("pause") - .withDescription("Wait on keyboard input after commit & batch close. default: disabled") - .withLongOpt("pause") - .create('x')); - - - Parser parser = new GnuParser(); - CommandLine cmdline = null; - try { - cmdline = parser.parse(options, args); - } catch (ParseException e) { - System.err.println(e.getMessage()); - usage(options); - } - - boolean pause = cmdline.hasOption('x'); - String db = cmdline.getOptionValue('d'); - String table = cmdline.getOptionValue('t'); - String uri = cmdline.getOptionValue('m'); - int txnsPerBatch = Integer.parseInt(cmdline.getOptionValue('n', "100")); - int writers = Integer.parseInt(cmdline.getOptionValue('w', "2")); - int batches = Integer.parseInt(cmdline.getOptionValue('i', "10")); - int recordsPerTxn = Integer.parseInt(cmdline.getOptionValue('r', "100")); - int frequency = Integer.parseInt(cmdline.getOptionValue('f', "1")); - int ap = Integer.parseInt(cmdline.getOptionValue('a', "5")); - float abortPct = ((float)ap) / 100.0f; - String[] partVals = cmdline.getOptionValues('p'); - String[] cols = cmdline.getOptionValues('c'); - String[] types = cmdline.getOptionValues('s'); - - StreamingIntegrationTester sit = new StreamingIntegrationTester(db, table, uri, - txnsPerBatch, writers, batches, recordsPerTxn, frequency, abortPct, partVals, cols, types - , pause); - sit.go(); - } - - static void usage(Options options) { - HelpFormatter hf = new HelpFormatter(); - hf.printHelp(HelpFormatter.DEFAULT_WIDTH, "sit [options]", "Usage: ", options, ""); - System.exit(-1); - } - - private String db; - private String table; - private String uri; - private int txnsPerBatch; - private int writers; - private int batches; - private int recordsPerTxn; - private int frequency; - private float abortPct; - private String[] partVals; - private String[] cols; - private String[] types; - private boolean pause; - - - private StreamingIntegrationTester(String db, String table, String uri, int txnsPerBatch, - int writers, int batches, int recordsPerTxn, - int frequency, float abortPct, String[] partVals, - String[] cols, String[] types, boolean pause) { - this.db = db; - this.table = table; - this.uri = uri; - this.txnsPerBatch = txnsPerBatch; - this.writers = writers; - this.batches = batches; - this.recordsPerTxn = recordsPerTxn; - this.frequency = frequency; - this.abortPct = abortPct; - this.partVals = partVals; - this.cols = cols; - this.types = types; - this.pause = pause; - } - - private void go() { - HiveEndPoint endPoint = null; - try { - if (partVals == null) { - endPoint = new HiveEndPoint(uri, db, table, null); - } else { - endPoint = new HiveEndPoint(uri, db, table, Arrays.asList(partVals)); - } - - for (int i = 0; i < writers; i++) { - Writer w = new Writer(endPoint, i, txnsPerBatch, batches, recordsPerTxn, frequency, abortPct, - cols, types, pause); - w.start(); - } - - } catch (Throwable t) { - System.err.println("Caught exception while testing: " + StringUtils.stringifyException(t)); - } - } - - private static class Writer extends Thread { - private HiveEndPoint endPoint; - private int txnsPerBatch; - private int batches; - private int writerNumber; - private int recordsPerTxn; - private int frequency; - private float abortPct; - private String[] cols; - private String[] types; - private boolean pause; - private Random rand; - - Writer(HiveEndPoint endPoint, int writerNumber, int txnsPerBatch, int batches, - int recordsPerTxn, int frequency, float abortPct, String[] cols, String[] types - , boolean pause) { - this.endPoint = endPoint; - this.txnsPerBatch = txnsPerBatch; - this.batches = batches; - this.writerNumber = writerNumber; - this.recordsPerTxn = recordsPerTxn; - this.frequency = frequency * 1000; - this.abortPct = abortPct; - this.cols = cols; - this.types = types; - this.pause = pause; - rand = new Random(); - } - - @Override - public void run() { - StreamingConnection conn = null; - try { - conn = endPoint.newConnection(true, "UT_" + Thread.currentThread().getName()); - RecordWriter writer = new DelimitedInputWriter(cols, ",", endPoint); - - for (int i = 0; i < batches; i++) { - long start = System.currentTimeMillis(); - LOG.info("Starting batch " + i); - TransactionBatch batch = conn.fetchTransactionBatch(txnsPerBatch, writer); - try { - while (batch.remainingTransactions() > 0) { - batch.beginNextTransaction(); - for (int j = 0; j < recordsPerTxn; j++) { - batch.write(generateRecord(cols, types)); - } - if (rand.nextFloat() < abortPct) batch.abort(); - else - batch.commit(); - if (pause) { - System.out.println("Writer " + writerNumber + - " committed... press Enter to continue. " + Thread.currentThread().getId()); - System.in.read(); - } - } - long end = System.currentTimeMillis(); - if (end - start < frequency) Thread.sleep(frequency - (end - start)); - } finally { - batch.close(); - if (pause) { - System.out.println("Writer " + writerNumber + - " has closed a Batch.. press Enter to continue. " + Thread.currentThread().getId()); - System.in.read(); - } - } - } - } catch (Throwable t) { - System.err.println("Writer number " + writerNumber - + " caught exception while testing: " + StringUtils.stringifyException(t)); - } finally { - if (conn!=null) conn.close(); - } - } - - private byte[] generateRecord(String[] cols, String[] types) { - // TODO make it so I can randomize the column order - - StringBuilder buf = new StringBuilder(); - for (int i = 0; i < types.length; i++) { - buf.append(generateColumn(types[i])); - buf.append(","); - } - return buf.toString().getBytes(); - } - - private String generateColumn(String type) { - if ("string".equals(type.toLowerCase())) { - return "When that Aprilis with his showers swoot"; - } else if (type.toLowerCase().startsWith("int")) { - return "42"; - } else if (type.toLowerCase().startsWith("dec") || type.toLowerCase().equals("float")) { - return "3.141592654"; - } else if (type.toLowerCase().equals("datetime")) { - return "2014-03-07 15:33:22"; - } else if (type.toLowerCase().equals("date")) { - return "1955-11-12"; - } else { - throw new RuntimeException("Sorry, I don't know the type " + type); - } - } - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestDelimitedInputWriter.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestDelimitedInputWriter.java deleted file mode 100644 index 32eead4e0a..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestDelimitedInputWriter.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; - -public class TestDelimitedInputWriter { - @Test - public void testFieldReordering() throws Exception { - - ArrayList colNames = Lists.newArrayList(new String[]{"col1", "col2", "col3", "col4", "col5"}); - {//1) test dropping fields - first middle & last - String[] fieldNames = {null, "col2", null, "col4", null}; - int[] mapping = DelimitedInputWriter.getFieldReordering(fieldNames, colNames); - Assert.assertTrue(Arrays.equals(mapping, new int[]{-1, 1, -1, 3, -1})); - } - - {//2) test reordering - String[] fieldNames = {"col5", "col4", "col3", "col2", "col1"}; - int[] mapping = DelimitedInputWriter.getFieldReordering(fieldNames, colNames); - Assert.assertTrue( Arrays.equals(mapping, new int[]{4,3,2,1,0}) ); - } - - {//3) test bad field names - String[] fieldNames = {"xyz", "abc", "col3", "col4", "as"}; - try { - DelimitedInputWriter.getFieldReordering(fieldNames, colNames); - Assert.fail(); - } catch (InvalidColumn e) { - // should throw - } - } - - {//4) test few field names - String[] fieldNames = {"col3", "col4"}; - int[] mapping = DelimitedInputWriter.getFieldReordering(fieldNames, colNames); - Assert.assertTrue( Arrays.equals(mapping, new int[]{2,3}) ); - } - - {//5) test extra field names - String[] fieldNames = {"col5", "col4", "col3", "col2", "col1", "col1"}; - try { - DelimitedInputWriter.getFieldReordering(fieldNames, colNames); - Assert.fail(); - } catch (InvalidColumn e) { - //show throw - } - } - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java deleted file mode 100644 index 569de706df..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java +++ /dev/null @@ -1,2373 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileFilter; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.PrintStream; -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hive.cli.CliSessionState; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.common.TableName; -import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.common.ValidWriteIdList; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.Validator; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse; -import org.apache.hadoop.hive.metastore.api.LockState; -import org.apache.hadoop.hive.metastore.api.LockType; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.ShowLocksRequest; -import org.apache.hadoop.hive.metastore.api.ShowLocksResponse; -import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement; -import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.hadoop.hive.metastore.api.TxnInfo; -import org.apache.hadoop.hive.metastore.api.TxnState; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService; -import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; -import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; -import org.apache.hadoop.hive.metastore.txn.TxnStore; -import org.apache.hadoop.hive.metastore.txn.TxnUtils; -import org.apache.hadoop.hive.ql.DriverFactory; -import org.apache.hadoop.hive.ql.IDriver; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.IOConstants; -import org.apache.hadoop.hive.ql.io.orc.OrcFile; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.hive.ql.io.orc.Reader; -import org.apache.hadoop.hive.ql.io.orc.RecordReader; -import org.apache.hadoop.hive.ql.processors.CommandProcessorException; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.txn.compactor.Worker; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableIntObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.orc.impl.OrcAcidUtils; -import org.apache.orc.tools.FileDump; -import org.apache.thrift.TException; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_COUNT; - - -public class TestStreaming { - private static final Logger LOG = LoggerFactory.getLogger(TestStreaming.class); - - public static class RawFileSystem extends RawLocalFileSystem { - private static final URI NAME; - static { - try { - NAME = new URI("raw:///"); - } catch (URISyntaxException se) { - throw new IllegalArgumentException("bad uri", se); - } - } - - @Override - public URI getUri() { - return NAME; - } - - @Override - public String getScheme() { - return "raw"; - } - - @Override - public FileStatus getFileStatus(Path path) throws IOException { - File file = pathToFile(path); - if (!file.exists()) { - throw new FileNotFoundException("Can't find " + path); - } - // get close enough - short mod = 0; - if (file.canRead()) { - mod |= 0444; - } - if (file.canWrite()) { - mod |= 0200; - } - if (file.canExecute()) { - mod |= 0111; - } - return new FileStatus(file.length(), file.isDirectory(), 1, 1024, - file.lastModified(), file.lastModified(), - FsPermission.createImmutable(mod), "owen", "users", path); - } - } - - private static final String COL1 = "id"; - private static final String COL2 = "msg"; - - private final HiveConf conf; - private IDriver driver; - private final IMetaStoreClient msClient; - - final String metaStoreURI = null; - - // partitioned table - private final static String dbName = "testing"; - private final static String tblName = "alerts"; - private final static String[] fieldNames = new String[]{COL1,COL2}; - List partitionVals; - private static Path partLoc; - private static Path partLoc2; - - // unpartitioned table - private final static String dbName2 = "testing2"; - private final static String tblName2 = "alerts"; - private final static String[] fieldNames2 = new String[]{COL1,COL2}; - - - // for bucket join testing - private final static String dbName3 = "testing3"; - private final static String tblName3 = "dimensionTable"; - private final static String dbName4 = "testing4"; - private final static String tblName4 = "factTable"; - List partitionVals2; - - - private final String PART1_CONTINENT = "Asia"; - private final String PART1_COUNTRY = "India"; - - @Rule - public TemporaryFolder dbFolder = new TemporaryFolder(); - - - public TestStreaming() throws Exception { - partitionVals = new ArrayList(2); - partitionVals.add(PART1_CONTINENT); - partitionVals.add(PART1_COUNTRY); - - partitionVals2 = new ArrayList(1); - partitionVals2.add(PART1_COUNTRY); - - - conf = new HiveConf(this.getClass()); - conf.set("fs.raw.impl", RawFileSystem.class.getName()); - conf - .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, - "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory"); - TxnDbUtil.setConfValues(conf); - if (metaStoreURI!=null) { - conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreURI); - } - conf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); - dbFolder.create(); - - - //1) Start from a clean slate (metastore) - TxnDbUtil.cleanDb(conf); - TxnDbUtil.prepDb(conf); - - //2) obtain metastore clients - msClient = new HiveMetaStoreClient(conf); - } - - @Before - public void setup() throws Exception { - SessionState.start(new CliSessionState(conf)); - driver = DriverFactory.newDriver(conf); - driver.setMaxRows(200002);//make sure Driver returns all results - // drop and recreate the necessary databases and tables - dropDB(msClient, dbName); - - String[] colNames = new String[] {COL1, COL2}; - String[] colTypes = new String[] {serdeConstants.INT_TYPE_NAME, serdeConstants.STRING_TYPE_NAME}; - String[] bucketCols = new String[] {COL1}; - String loc1 = dbFolder.newFolder(dbName + ".db").toString(); - String[] partNames = new String[]{"Continent", "Country"}; - partLoc = createDbAndTable(driver, dbName, tblName, partitionVals, colNames, colTypes, bucketCols, partNames, loc1, 1); - - dropDB(msClient, dbName2); - String loc2 = dbFolder.newFolder(dbName2 + ".db").toString(); - partLoc2 = createDbAndTable(driver, dbName2, tblName2, null, colNames, colTypes, bucketCols, null, loc2, 2); - - String loc3 = dbFolder.newFolder("testing5.db").toString(); - createStoreSales("testing5", loc3); - - runDDL(driver, "drop table testBucketing3.streamedtable"); - runDDL(driver, "drop table testBucketing3.finaltable"); - runDDL(driver, "drop table testBucketing3.nobucket"); - } - - @After - public void cleanup() throws Exception { - msClient.close(); - driver.close(); - } - - private static List getPartitionKeys() { - List fields = new ArrayList(); - // Defining partition names in unsorted order - fields.add(new FieldSchema("continent", serdeConstants.STRING_TYPE_NAME, "")); - fields.add(new FieldSchema("country", serdeConstants.STRING_TYPE_NAME, "")); - return fields; - } - - private void createStoreSales(String dbName, String loc) throws Exception { - String dbUri = "raw://" + new Path(loc).toUri().toString(); - String tableLoc = dbUri + Path.SEPARATOR + "store_sales"; - - boolean success = runDDL(driver, "create database IF NOT EXISTS " + dbName + " location '" + dbUri + "'"); - Assert.assertTrue(success); - success = runDDL(driver, "use " + dbName); - Assert.assertTrue(success); - - success = runDDL(driver, "drop table if exists store_sales"); - Assert.assertTrue(success); - success = runDDL(driver, "create table store_sales\n" + - "(\n" + - " ss_sold_date_sk int,\n" + - " ss_sold_time_sk int,\n" + - " ss_item_sk int,\n" + - " ss_customer_sk int,\n" + - " ss_cdemo_sk int,\n" + - " ss_hdemo_sk int,\n" + - " ss_addr_sk int,\n" + - " ss_store_sk int,\n" + - " ss_promo_sk int,\n" + - " ss_ticket_number int,\n" + - " ss_quantity int,\n" + - " ss_wholesale_cost decimal(7,2),\n" + - " ss_list_price decimal(7,2),\n" + - " ss_sales_price decimal(7,2),\n" + - " ss_ext_discount_amt decimal(7,2),\n" + - " ss_ext_sales_price decimal(7,2),\n" + - " ss_ext_wholesale_cost decimal(7,2),\n" + - " ss_ext_list_price decimal(7,2),\n" + - " ss_ext_tax decimal(7,2),\n" + - " ss_coupon_amt decimal(7,2),\n" + - " ss_net_paid decimal(7,2),\n" + - " ss_net_paid_inc_tax decimal(7,2),\n" + - " ss_net_profit decimal(7,2)\n" + - ")\n" + - " partitioned by (dt string)\n" + - "clustered by (ss_store_sk, ss_promo_sk)\n" + - "INTO 4 BUCKETS stored as orc " + " location '" + tableLoc + "'" + " TBLPROPERTIES ('orc.compress'='NONE', 'transactional'='true')"); - Assert.assertTrue(success); - - success = runDDL(driver, "alter table store_sales add partition(dt='2015')"); - Assert.assertTrue(success); - } - /** - * make sure it works with table where bucket col is not 1st col - * @throws Exception - */ - @Test - public void testBucketingWhereBucketColIsNotFirstCol() throws Exception { - List partitionVals = new ArrayList(); - partitionVals.add("2015"); - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testing5", "store_sales", partitionVals); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"ss_sold_date_sk","ss_sold_time_sk", "ss_item_sk", - "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", - "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", - "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"},",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - - StringBuilder row = new StringBuilder(); - for(int i = 0; i < 10; i++) { - for(int ints = 0; ints < 11; ints++) { - row.append(ints).append(','); - } - for(int decs = 0; decs < 12; decs++) { - row.append(i + 0.1).append(','); - } - row.setLength(row.length() - 1); - txnBatch.write(row.toString().getBytes()); - } - txnBatch.commit(); - txnBatch.close(); - connection.close(); - - ArrayList res = queryTable(driver, "select row__id.bucketid, * from testing5.store_sales"); - for (String re : res) { - System.out.println(re); - } - } - - /** - * Test that streaming can write to unbucketed table. - */ - @Test - public void testNoBuckets() throws Exception { - queryTable(driver, "drop table if exists default.streamingnobuckets"); - //todo: why does it need transactional_properties? - queryTable(driver, "create table default.streamingnobuckets (a string, b string) stored as orc TBLPROPERTIES('transactional'='true', 'transactional_properties'='default')"); - queryTable(driver, "insert into default.streamingnobuckets values('foo','bar')"); - List rs = queryTable(driver, "select * from default.streamingNoBuckets"); - Assert.assertEquals(1, rs.size()); - Assert.assertEquals("foo\tbar", rs.get(0)); - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "Default", "StreamingNoBuckets", null); - String[] colNames1 = new String[] { "a", "b" }; - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter wr = new DelimitedInputWriter(colNames1,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, wr); - txnBatch.beginNextTransaction(); - txnBatch.write("a1,b2".getBytes()); - txnBatch.write("a3,b4".getBytes()); - TxnStore txnHandler = TxnUtils.getTxnStore(conf); - ShowLocksResponse resp = txnHandler.showLocks(new ShowLocksRequest()); - Assert.assertEquals(resp.getLocksSize(), 1); - Assert.assertEquals("streamingnobuckets", resp.getLocks().get(0).getTablename()); - Assert.assertEquals("default", resp.getLocks().get(0).getDbname()); - txnBatch.commit(); - txnBatch.beginNextTransaction(); - txnBatch.write("a5,b6".getBytes()); - txnBatch.write("a7,b8".getBytes()); - txnBatch.commit(); - txnBatch.close(); - - Assert.assertEquals("", 0, BucketCodec.determineVersion(536870912).decodeWriterId(536870912)); - rs = queryTable(driver,"select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); - - Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/delta_0000001_0000001_0000/bucket_00000_0")); - Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":0}\ta1\tb2")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); - Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); - Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); - Assert.assertTrue(rs.get(4), rs.get(4).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":1}\ta7\tb8")); - Assert.assertTrue(rs.get(4), rs.get(4).endsWith("streamingnobuckets/delta_0000002_0000003/bucket_00000")); - - queryTable(driver, "update default.streamingnobuckets set a=0, b=0 where a='a7'"); - queryTable(driver, "delete from default.streamingnobuckets where a='a1'"); - rs = queryTable(driver, "select a, b from default.streamingnobuckets order by a, b"); - int row = 0; - Assert.assertEquals("at row=" + row, "0\t0", rs.get(row++)); - Assert.assertEquals("at row=" + row, "a3\tb4", rs.get(row++)); - Assert.assertEquals("at row=" + row, "a5\tb6", rs.get(row++)); - Assert.assertEquals("at row=" + row, "foo\tbar", rs.get(row++)); - - queryTable(driver, "alter table default.streamingnobuckets compact 'major'"); - runWorker(conf); - rs = queryTable(driver,"select ROW__ID, a, b, INPUT__FILE__NAME from default.streamingnobuckets order by ROW__ID"); - - Assert.assertTrue(rs.get(0), rs.get(0).startsWith("{\"writeid\":1,\"bucketid\":536870912,\"rowid\":0}\tfoo\tbar")); - Assert.assertTrue(rs.get(0), rs.get(0).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); - Assert.assertTrue(rs.get(1), rs.get(1).startsWith("{\"writeid\":2,\"bucketid\":536870912,\"rowid\":1}\ta3\tb4")); - Assert.assertTrue(rs.get(1), rs.get(1).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); - Assert.assertTrue(rs.get(2), rs.get(2).startsWith("{\"writeid\":3,\"bucketid\":536870912,\"rowid\":0}\ta5\tb6")); - Assert.assertTrue(rs.get(2), rs.get(2).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); - Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\t0\t0")); - Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005_v0000025/bucket_00000")); - } - - /** - * this is a clone from TestTxnStatement2.... - */ - public static void runWorker(HiveConf hiveConf) throws Exception { - AtomicBoolean stop = new AtomicBoolean(true); - Worker t = new Worker(); - t.setThreadId((int) t.getId()); - t.setConf(hiveConf); - AtomicBoolean looped = new AtomicBoolean(); - t.init(stop, looped); - t.run(); - } - - // stream data into streaming table with N buckets, then copy the data into another bucketed table - // check if bucketing in both was done in the same way - @Test - @Ignore - public void testStreamBucketingMatchesRegularBucketing() throws Exception { - int bucketCount = 100; - - String dbUri = "raw://" + new Path(dbFolder.newFolder().toString()).toUri().toString(); - String tableLoc = "'" + dbUri + Path.SEPARATOR + "streamedtable" + "'"; - String tableLoc2 = "'" + dbUri + Path.SEPARATOR + "finaltable" + "'"; - String tableLoc3 = "'" + dbUri + Path.SEPARATOR + "nobucket" + "'"; - try (IDriver driver = DriverFactory.newDriver(conf)) { - runDDL(driver, "create database testBucketing3"); - runDDL(driver, "use testBucketing3"); - runDDL(driver, "create table streamedtable ( key1 string,key2 int,data string ) clustered by ( key1,key2 ) into " - + bucketCount + " buckets stored as orc location " + tableLoc + " TBLPROPERTIES ('transactional'='true')"); - // In 'nobucket' table we capture bucketid from streamedtable to workaround a hive bug that prevents joins two identically bucketed tables - runDDL(driver, "create table nobucket ( bucketid int, key1 string,key2 int,data string ) location " + tableLoc3); - runDDL(driver, - "create table finaltable ( bucketid int, key1 string,key2 int,data string ) clustered by ( key1,key2 ) into " - + bucketCount + " buckets stored as orc location " + tableLoc2 + " TBLPROPERTIES ('transactional'='true')"); - - - String[] records = new String[]{ - "PSFAHYLZVC,29,EPNMA", - "PPPRKWAYAU,96,VUTEE", - "MIAOFERCHI,3,WBDSI", - "CEGQAZOWVN,0,WCUZL", - "XWAKMNSVQF,28,YJVHU", - "XBWTSAJWME,2,KDQFO", - "FUVLQTAXAY,5,LDSDG", - "QTQMDJMGJH,6,QBOMA", - "EFLOTLWJWN,71,GHWPS", - "PEQNAOJHCM,82,CAAFI", - "MOEKQLGZCP,41,RUACR", - "QZXMCOPTID,37,LFLWE", - "EYALVWICRD,13,JEZLC", - "VYWLZAYTXX,16,DMVZX", - "OSALYSQIXR,47,HNZVE", - "JGKVHKCEGQ,25,KSCJB", - "WQFMMYDHET,12,DTRWA", - "AJOVAYZKZQ,15,YBKFO", - "YAQONWCUAU,31,QJNHZ", - "DJBXUEUOEB,35,IYCBL" - }; - - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "streamedtable", null); - String[] colNames1 = new String[]{"key1", "key2", "data"}; - DelimitedInputWriter wr = new DelimitedInputWriter(colNames1, ",", endPt); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, wr); - txnBatch.beginNextTransaction(); - - for (String record : records) { - txnBatch.write(record.toString().getBytes()); - } - - txnBatch.commit(); - txnBatch.close(); - connection.close(); - - ArrayList res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2"); - for (String re : res1) { - System.out.println(re); - } - - driver.run("insert into nobucket select row__id.bucketid,* from streamedtable"); - runDDL(driver, " insert into finaltable select * from nobucket"); - ArrayList res2 = queryTable(driver, - "select row__id.bucketid,* from finaltable where row__id.bucketid<>bucketid"); - for (String s : res2) { - LOG.error(s); - } - Assert.assertTrue(res2.isEmpty()); - } finally { - conf.unset(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED.varname); - } - } - - - @Test - public void testTableValidation() throws Exception { - int bucketCount = 100; - - String dbUri = "raw://" + new Path(dbFolder.newFolder().toString()).toUri().toString(); - String tbl1 = "validation1"; - String tbl2 = "validation2"; - - String tableLoc = "'" + dbUri + Path.SEPARATOR + tbl1 + "'"; - String tableLoc2 = "'" + dbUri + Path.SEPARATOR + tbl2 + "'"; - - runDDL(driver, "create database testBucketing3"); - runDDL(driver, "use testBucketing3"); - - runDDL(driver, "create table " + tbl1 + " ( key1 string, data string ) clustered by ( key1 ) into " - + bucketCount + " buckets stored as orc location " + tableLoc + " TBLPROPERTIES ('transactional'='false')") ; - - runDDL(driver, "create table " + tbl2 + " ( key1 string, data string ) clustered by ( key1 ) into " - + bucketCount + " buckets stored as orc location " + tableLoc2 + " TBLPROPERTIES ('transactional'='false')") ; - - - try { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "validation1", null); - endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - Assert.assertTrue("InvalidTable exception was not thrown", false); - } catch (InvalidTable e) { - // expecting this exception - } - try { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "validation2", null); - endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - Assert.assertTrue("InvalidTable exception was not thrown", false); - } catch (InvalidTable e) { - // expecting this exception - } - } - - /** - * @deprecated use {@link #checkDataWritten2(Path, long, long, int, String, boolean, String...)} - - * there is little value in using InputFormat directly - */ - @Deprecated - private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int buckets, int numExpectedFiles, - String... records) throws Exception { - ValidWriteIdList writeIds = getTransactionContext(conf); - AcidUtils.Directory dir = AcidUtils.getAcidState(null, partitionPath, conf, writeIds, null, false, null, false); - Assert.assertEquals(0, dir.getObsolete().size()); - Assert.assertEquals(0, dir.getOriginalFiles().size()); - List current = dir.getCurrentDirectories(); - System.out.println("Files found: "); - for (AcidUtils.ParsedDelta pd : current) { - System.out.println(pd.getPath().toString()); - } - Assert.assertEquals(numExpectedFiles, current.size()); - - // find the absolute minimum transaction - long min = Long.MAX_VALUE; - long max = Long.MIN_VALUE; - for (AcidUtils.ParsedDelta pd : current) { - if (pd.getMaxWriteId() > max) { - max = pd.getMaxWriteId(); - } - if (pd.getMinWriteId() < min) { - min = pd.getMinWriteId(); - } - } - Assert.assertEquals(minTxn, min); - Assert.assertEquals(maxTxn, max); - - InputFormat inf = new OrcInputFormat(); - JobConf job = new JobConf(); - job.set("mapred.input.dir", partitionPath.toString()); - job.set(BUCKET_COUNT, Integer.toString(buckets)); - job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, "id,msg"); - job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, "bigint:string"); - AcidUtils.setAcidOperationalProperties(job, true, null); - job.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); - job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.toString()); - job.set(ValidTxnList.VALID_TXNS_KEY, conf.get(ValidTxnList.VALID_TXNS_KEY)); - InputSplit[] splits = inf.getSplits(job, buckets); - Assert.assertEquals(numExpectedFiles, splits.length); - org.apache.hadoop.mapred.RecordReader rr = - inf.getRecordReader(splits[0], job, Reporter.NULL); - - NullWritable key = rr.createKey(); - OrcStruct value = rr.createValue(); - for (String record : records) { - Assert.assertEquals(true, rr.next(key, value)); - Assert.assertEquals(record, value.toString()); - } - Assert.assertEquals(false, rr.next(key, value)); - } - /** - * @param validationQuery query to read from table to compare data against {@code records} - * @param records expected data. each row is CVS list of values - */ - private void checkDataWritten2(Path partitionPath, long minTxn, long maxTxn, int numExpectedFiles, - String validationQuery, boolean vectorize, String... records) throws Exception { - AcidUtils.Directory dir = AcidUtils.getAcidState(null, partitionPath, conf, getTransactionContext(conf), null, - false, null, false); - Assert.assertEquals(0, dir.getObsolete().size()); - Assert.assertEquals(0, dir.getOriginalFiles().size()); - List current = dir.getCurrentDirectories(); - System.out.println("Files found: "); - for (AcidUtils.ParsedDelta pd : current) { - System.out.println(pd.getPath().toString()); - } - Assert.assertEquals(numExpectedFiles, current.size()); - - // find the absolute minimum transaction - long min = Long.MAX_VALUE; - long max = Long.MIN_VALUE; - for (AcidUtils.ParsedDelta pd : current) { - if (pd.getMaxWriteId() > max) { - max = pd.getMaxWriteId(); - } - if (pd.getMinWriteId() < min) { - min = pd.getMinWriteId(); - } - } - Assert.assertEquals(minTxn, min); - Assert.assertEquals(maxTxn, max); - boolean isVectorizationEnabled = conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED); - if(vectorize) { - conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, true); - } - - String currStrategy = conf.getVar(HiveConf.ConfVars.HIVE_ORC_SPLIT_STRATEGY); - for(String strategy : ((Validator.StringSet)HiveConf.ConfVars.HIVE_ORC_SPLIT_STRATEGY.getValidator()).getExpected()) { - //run it with each split strategy - make sure there are differences - conf.setVar(HiveConf.ConfVars.HIVE_ORC_SPLIT_STRATEGY, strategy.toUpperCase()); - List actualResult = queryTable(driver, validationQuery); - for (int i = 0; i < actualResult.size(); i++) { - Assert.assertEquals("diff at [" + i + "]. actual=" + actualResult + " expected=" + - Arrays.toString(records), records[i], actualResult.get(i)); - } - } - conf.setVar(HiveConf.ConfVars.HIVE_ORC_SPLIT_STRATEGY, currStrategy); - conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, isVectorizationEnabled); - } - - private ValidWriteIdList getTransactionContext(Configuration conf) throws Exception { - ValidTxnList validTxnList = msClient.getValidTxns(); - conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); - List v = msClient.getValidWriteIds(Collections - .singletonList(TableName.getDbTable(dbName, tblName)), validTxnList.writeToString()); - return TxnCommonUtils.createValidReaderWriteIdList(v.get(0)); - } - private void checkNothingWritten(Path partitionPath) throws Exception { - AcidUtils.Directory dir = AcidUtils.getAcidState(null, partitionPath, conf, getTransactionContext(conf), null, - false, null, false); - Assert.assertEquals(0, dir.getObsolete().size()); - Assert.assertEquals(0, dir.getOriginalFiles().size()); - List current = dir.getCurrentDirectories(); - Assert.assertEquals(0, current.size()); - } - - @Test - public void testEndpointConnection() throws Exception { - // For partitioned table, partitionVals are specified - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, partitionVals); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); //shouldn't throw - connection.close(); - - // For unpartitioned table, partitionVals are not specified - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - endPt.newConnection(false, "UT_" + Thread.currentThread().getName()).close(); // should not throw - - // For partitioned table, partitionVals are not specified - try { - endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, null); - connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - Assert.assertTrue("ConnectionError was not thrown", false); - connection.close(); - } catch (ConnectionError e) { - // expecting this exception - String errMsg = "doesn't specify any partitions for partitioned table"; - Assert.assertTrue(e.toString().endsWith(errMsg)); - } - - // For unpartitioned table, partition values are specified - try { - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, partitionVals); - connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - Assert.assertTrue("ConnectionError was not thrown", false); - connection.close(); - } catch (ConnectionError e) { - // expecting this exception - String errMsg = "specifies partitions for unpartitioned table"; - Assert.assertTrue(e.toString().endsWith(errMsg)); - } - } - - @Test - public void testAddPartition() throws Exception { - List newPartVals = new ArrayList(2); - newPartVals.add(PART1_CONTINENT); - newPartVals.add("Nepal"); - - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName - , newPartVals); - - // Ensure partition is absent - try { - msClient.getPartition(endPt.database, endPt.table, endPt.partitionVals); - Assert.assertTrue("Partition already exists", false); - } catch (NoSuchObjectException e) { - // expect this exception - } - - // Create partition - Assert.assertNotNull(endPt.newConnection(true, "UT_" + Thread.currentThread().getName())); - - // Ensure partition is present - Partition p = msClient.getPartition(endPt.database, endPt.table, endPt.partitionVals); - Assert.assertNotNull("Did not find added partition", p); - } - - @Test - public void testTransactionBatchEmptyCommit() throws Exception { - // 1) to partitioned table - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - - txnBatch.beginNextTransaction(); - txnBatch.commit(); - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - txnBatch.close(); - connection.close(); - - // 2) To unpartitioned table - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - writer = new DelimitedInputWriter(fieldNames2,",", endPt); - connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.commit(); - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - txnBatch.close(); - connection.close(); - } - - /** - * check that transactions that have not heartbeated and timedout get properly aborted - * @throws Exception - */ - @Test - public void testTimeOutReaper() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames2,",", endPt); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(5, writer); - txnBatch.beginNextTransaction(); - conf.setTimeVar(HiveConf.ConfVars.HIVE_TIMEDOUT_TXN_REAPER_START, 0, TimeUnit.SECONDS); - //ensure txn timesout - conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1, TimeUnit.MILLISECONDS); - AcidHouseKeeperService houseKeeperService = new AcidHouseKeeperService(); - houseKeeperService.setConf(conf); - houseKeeperService.run(); - try { - //should fail because the TransactionBatch timed out - txnBatch.commit(); - } - catch(TransactionError e) { - Assert.assertTrue("Expected aborted transaction", e.getCause() instanceof TxnAbortedException); - } - txnBatch.close(); - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.commit(); - txnBatch.beginNextTransaction(); - houseKeeperService.run(); - try { - //should fail because the TransactionBatch timed out - txnBatch.commit(); - } - catch(TransactionError e) { - Assert.assertTrue("Expected aborted transaction", e.getCause() instanceof TxnAbortedException); - } - txnBatch.close(); - connection.close(); - } - - @Test - public void testHeartbeat() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames2,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(5, writer); - txnBatch.beginNextTransaction(); - //todo: this should ideally check Transaction heartbeat as well, but heartbeat - //timestamp is not reported yet - //GetOpenTxnsInfoResponse txnresp = msClient.showTxns(); - ShowLocksRequest request = new ShowLocksRequest(); - request.setDbname(dbName2); - request.setTablename(tblName2); - ShowLocksResponse response = msClient.showLocks(request); - Assert.assertEquals("Wrong nubmer of locks: " + response, 1, response.getLocks().size()); - ShowLocksResponseElement lock = response.getLocks().get(0); - long acquiredAt = lock.getAcquiredat(); - long heartbeatAt = lock.getLastheartbeat(); - txnBatch.heartbeat(); - response = msClient.showLocks(request); - Assert.assertEquals("Wrong number of locks2: " + response, 1, response.getLocks().size()); - lock = response.getLocks().get(0); - Assert.assertEquals("Acquired timestamp didn't match", acquiredAt, lock.getAcquiredat()); - Assert.assertTrue("Expected new heartbeat (" + lock.getLastheartbeat() + - ") == old heartbeat(" + heartbeatAt +")", lock.getLastheartbeat() == heartbeatAt); - txnBatch.close(); - int txnBatchSize = 200; - txnBatch = connection.fetchTransactionBatch(txnBatchSize, writer); - for(int i = 0; i < txnBatchSize; i++) { - txnBatch.beginNextTransaction(); - if(i % 47 == 0) { - txnBatch.heartbeat(); - } - if(i % 10 == 0) { - txnBatch.abort(); - } - else { - txnBatch.commit(); - } - if(i % 37 == 0) { - txnBatch.heartbeat(); - } - } - - } - @Test - public void testTransactionBatchEmptyAbort() throws Exception { - // 1) to partitioned table - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.abort(); - Assert.assertEquals(TransactionBatch.TxnState.ABORTED - , txnBatch.getCurrentTransactionState()); - txnBatch.close(); - connection.close(); - - // 2) to unpartitioned table - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - writer = new DelimitedInputWriter(fieldNames,",", endPt); - connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.abort(); - Assert.assertEquals(TransactionBatch.TxnState.ABORTED - , txnBatch.getCurrentTransactionState()); - txnBatch.close(); - connection.close(); - } - - @Test - public void testTransactionBatchCommit_Delimited() throws Exception { - testTransactionBatchCommit_Delimited(null); - } - @Test - public void testTransactionBatchCommit_DelimitedUGI() throws Exception { - testTransactionBatchCommit_Delimited(Utils.getUGI()); - } - private void testTransactionBatchCommit_Delimited(UserGroupInformation ugi) throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(true, conf, ugi, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt, conf, connection); - - // 1st Txn - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}"); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - - // 2nd Txn - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("2,Welcome to streaming".getBytes()); - - // data should not be visible - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}"); - - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}", - "{2, Welcome to streaming}"); - - txnBatch.close(); - Assert.assertEquals(TransactionBatch.TxnState.INACTIVE - , txnBatch.getCurrentTransactionState()); - - - connection.close(); - - - // To Unpartitioned table - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - connection = endPt.newConnection(true, conf, ugi, "UT_" + Thread.currentThread().getName()); - writer = new DelimitedInputWriter(fieldNames,",", endPt, conf, connection); - - // 1st Txn - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.commit(); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - connection.close(); - } - - @Test - public void testTransactionBatchCommit_Regex() throws Exception { - testTransactionBatchCommit_Regex(null); - } - @Test - public void testTransactionBatchCommit_RegexUGI() throws Exception { - testTransactionBatchCommit_Regex(Utils.getUGI()); - } - private void testTransactionBatchCommit_Regex(UserGroupInformation ugi) throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(true, conf, ugi, "UT_" + Thread.currentThread().getName()); - String regex = "([^,]*),(.*)"; - StrictRegexWriter writer = new StrictRegexWriter(regex, endPt, conf, connection); - - // 1st Txn - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}"); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - - // 2nd Txn - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("2,Welcome to streaming".getBytes()); - - // data should not be visible - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}"); - - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}", - "{2, Welcome to streaming}"); - - txnBatch.close(); - Assert.assertEquals(TransactionBatch.TxnState.INACTIVE - , txnBatch.getCurrentTransactionState()); - - - connection.close(); - - - // To Unpartitioned table - endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, null); - connection = endPt.newConnection(true, conf, ugi, "UT_" + Thread.currentThread().getName()); - regex = "([^:]*):(.*)"; - writer = new StrictRegexWriter(regex, endPt, conf, connection); - - // 1st Txn - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write("1:Hello streaming".getBytes()); - txnBatch.commit(); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - connection.close(); - } - - @Test - public void testTransactionBatchCommit_Json() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - StrictJsonWriter writer = new StrictJsonWriter(endPt, connection); - - // 1st Txn - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - String rec1 = "{\"id\" : 1, \"msg\": \"Hello streaming\"}"; - txnBatch.write(rec1.getBytes()); - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}"); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - - txnBatch.close(); - Assert.assertEquals(TransactionBatch.TxnState.INACTIVE - , txnBatch.getCurrentTransactionState()); - - connection.close(); - List rs = queryTable(driver, "select * from " + dbName + "." + tblName); - Assert.assertEquals(1, rs.size()); - } - - @Test - public void testRemainingTransactions() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt); - StreamingConnection connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - - // 1) test with txn.Commit() - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - int batch=0; - int initialCount = txnBatch.remainingTransactions(); - while (txnBatch.remainingTransactions()>0) { - txnBatch.beginNextTransaction(); - Assert.assertEquals(--initialCount, txnBatch.remainingTransactions()); - for (int rec=0; rec<2; ++rec) { - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write((batch * rec + ",Hello streaming").getBytes()); - } - txnBatch.commit(); - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - ++batch; - } - Assert.assertEquals(0, txnBatch.remainingTransactions()); - txnBatch.close(); - - Assert.assertEquals(TransactionBatch.TxnState.INACTIVE - , txnBatch.getCurrentTransactionState()); - - // 2) test with txn.Abort() - txnBatch = connection.fetchTransactionBatch(10, writer); - batch=0; - initialCount = txnBatch.remainingTransactions(); - while (txnBatch.remainingTransactions()>0) { - txnBatch.beginNextTransaction(); - Assert.assertEquals(--initialCount,txnBatch.remainingTransactions()); - for (int rec=0; rec<2; ++rec) { - Assert.assertEquals(TransactionBatch.TxnState.OPEN - , txnBatch.getCurrentTransactionState()); - txnBatch.write((batch * rec + ",Hello streaming").getBytes()); - } - txnBatch.abort(); - Assert.assertEquals(TransactionBatch.TxnState.ABORTED - , txnBatch.getCurrentTransactionState()); - ++batch; - } - Assert.assertEquals(0, txnBatch.remainingTransactions()); - txnBatch.close(); - - Assert.assertEquals(TransactionBatch.TxnState.INACTIVE - , txnBatch.getCurrentTransactionState()); - - connection.close(); - } - - @Test - public void testTransactionBatchAbort() throws Exception { - - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt, connection); - - - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.write("2,Welcome to streaming".getBytes()); - txnBatch.abort(); - - checkNothingWritten(partLoc); - - Assert.assertEquals(TransactionBatch.TxnState.ABORTED - , txnBatch.getCurrentTransactionState()); - - txnBatch.close(); - connection.close(); - - checkNothingWritten(partLoc); - - } - - - @Test - public void testTransactionBatchAbortAndCommit() throws Exception { - String agentInfo = "UT_" + Thread.currentThread().getName(); - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - StreamingConnection connection = endPt.newConnection(false, agentInfo); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.write("2,Welcome to streaming".getBytes()); - ShowLocksResponse resp = msClient.showLocks(new ShowLocksRequest()); - Assert.assertEquals("LockCount", 1, resp.getLocksSize()); - Assert.assertEquals("LockType", LockType.SHARED_READ, resp.getLocks().get(0).getType()); - Assert.assertEquals("LockState", LockState.ACQUIRED, resp.getLocks().get(0).getState()); - Assert.assertEquals("AgentInfo", agentInfo, resp.getLocks().get(0).getAgentInfo()); - txnBatch.abort(); - - checkNothingWritten(partLoc); - - Assert.assertEquals(TransactionBatch.TxnState.ABORTED - , txnBatch.getCurrentTransactionState()); - - txnBatch.beginNextTransaction(); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.write("2,Welcome to streaming".getBytes()); - txnBatch.commit(); - - checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}", - "{2, Welcome to streaming}"); - - txnBatch.close(); - connection.close(); - } - - @Test - public void testMultipleTransactionBatchCommits() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames,",", endPt); - StreamingConnection connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName()); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("1,Hello streaming".getBytes()); - txnBatch.commit(); - String validationQuery = "select id, msg from " + dbName + "." + tblName + " order by id, msg"; - checkDataWritten2(partLoc, 1, 10, 1, validationQuery, false, "1\tHello streaming"); - - txnBatch.beginNextTransaction(); - txnBatch.write("2,Welcome to streaming".getBytes()); - txnBatch.commit(); - - checkDataWritten2(partLoc, 1, 10, 1, validationQuery, true, "1\tHello streaming", - "2\tWelcome to streaming"); - - txnBatch.close(); - - // 2nd Txn Batch - txnBatch = connection.fetchTransactionBatch(10, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("3,Hello streaming - once again".getBytes()); - txnBatch.commit(); - - checkDataWritten2(partLoc, 1, 20, 2, validationQuery, false, "1\tHello streaming", - "2\tWelcome to streaming", "3\tHello streaming - once again"); - - txnBatch.beginNextTransaction(); - txnBatch.write("4,Welcome to streaming - once again".getBytes()); - txnBatch.commit(); - - checkDataWritten2(partLoc, 1, 20, 2, validationQuery, true, "1\tHello streaming", - "2\tWelcome to streaming", "3\tHello streaming - once again", - "4\tWelcome to streaming - once again"); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch.getCurrentTransactionState()); - - txnBatch.close(); - - connection.close(); - } - - @Test - public void testInterleavedTransactionBatchCommits() throws Exception { - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName, tblName, - partitionVals); - DelimitedInputWriter writer = new DelimitedInputWriter(fieldNames, ",", endPt); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - - // Acquire 1st Txn Batch - TransactionBatch txnBatch1 = connection.fetchTransactionBatch(10, writer); - txnBatch1.beginNextTransaction(); - - // Acquire 2nd Txn Batch - DelimitedInputWriter writer2 = new DelimitedInputWriter(fieldNames, ",", endPt); - TransactionBatch txnBatch2 = connection.fetchTransactionBatch(10, writer2); - txnBatch2.beginNextTransaction(); - - // Interleaved writes to both batches - txnBatch1.write("1,Hello streaming".getBytes()); - txnBatch2.write("3,Hello streaming - once again".getBytes()); - - checkNothingWritten(partLoc); - - txnBatch2.commit(); - - String validationQuery = "select id, msg from " + dbName + "." + tblName + " order by id, msg"; - checkDataWritten2(partLoc, 11, 20, 1, - validationQuery, true, "3\tHello streaming - once again"); - - txnBatch1.commit(); - /*now both batches have committed (but not closed) so we for each primary file we expect a side - file to exist and indicate the true length of primary file*/ - FileSystem fs = partLoc.getFileSystem(conf); - AcidUtils.Directory dir = AcidUtils.getAcidState(fs, partLoc, conf, getTransactionContext(conf), null, false, null, false); - for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { - for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { - Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath()); - Assert.assertTrue(lengthFile + " missing", fs.exists(lengthFile)); - long lengthFileSize = fs.getFileStatus(lengthFile).getLen(); - Assert.assertTrue("Expected " + lengthFile + " to be non empty. lengh=" + - lengthFileSize, lengthFileSize > 0); - long logicalLength = AcidUtils.getLogicalLength(fs, stat); - long actualLength = stat.getLen(); - Assert.assertTrue("", logicalLength == actualLength); - } - } - checkDataWritten2(partLoc, 1, 20, 2, - validationQuery, false,"1\tHello streaming", "3\tHello streaming - once again"); - - txnBatch1.beginNextTransaction(); - txnBatch1.write("2,Welcome to streaming".getBytes()); - - txnBatch2.beginNextTransaction(); - txnBatch2.write("4,Welcome to streaming - once again".getBytes()); - //here each batch has written data and committed (to bucket0 since table only has 1 bucket) - //so each of 2 deltas has 1 bucket0 and 1 bucket0_flush_length. Furthermore, each bucket0 - //has now received more data(logically - it's buffered) but it is not yet committed. - //lets check that side files exist, etc - dir = AcidUtils.getAcidState(fs, partLoc, conf, getTransactionContext(conf), null, false, null, false); - for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) { - for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) { - Path lengthFile = OrcAcidUtils.getSideFile(stat.getPath()); - Assert.assertTrue(lengthFile + " missing", fs.exists(lengthFile)); - long lengthFileSize = fs.getFileStatus(lengthFile).getLen(); - Assert.assertTrue("Expected " + lengthFile + " to be non empty. lengh=" + - lengthFileSize, lengthFileSize > 0); - long logicalLength = AcidUtils.getLogicalLength(fs, stat); - long actualLength = stat.getLen(); - Assert.assertTrue("", logicalLength <= actualLength); - } - } - checkDataWritten2(partLoc, 1, 20, 2, - validationQuery, true,"1\tHello streaming", "3\tHello streaming - once again"); - - txnBatch1.commit(); - - checkDataWritten2(partLoc, 1, 20, 2, - validationQuery, false, "1\tHello streaming", - "2\tWelcome to streaming", - "3\tHello streaming - once again"); - - txnBatch2.commit(); - - checkDataWritten2(partLoc, 1, 20, 2, - validationQuery, true, "1\tHello streaming", - "2\tWelcome to streaming", - "3\tHello streaming - once again", - "4\tWelcome to streaming - once again"); - - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch1.getCurrentTransactionState()); - Assert.assertEquals(TransactionBatch.TxnState.COMMITTED - , txnBatch2.getCurrentTransactionState()); - - txnBatch1.close(); - txnBatch2.close(); - - connection.close(); - } - - private static class WriterThd extends Thread { - - private final StreamingConnection conn; - private final DelimitedInputWriter writer; - private final String data; - private Throwable error; - - WriterThd(HiveEndPoint ep, String data) throws Exception { - super("Writer_" + data); - writer = new DelimitedInputWriter(fieldNames, ",", ep); - conn = ep.newConnection(false, "UT_" + Thread.currentThread().getName()); - this.data = data; - setUncaughtExceptionHandler(new UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread thread, Throwable throwable) { - error = throwable; - LOG.error("Thread " + thread.getName() + " died: " + throwable.getMessage(), throwable); - } - }); - } - - @Override - public void run() { - TransactionBatch txnBatch = null; - try { - txnBatch = conn.fetchTransactionBatch(10, writer); - while (txnBatch.remainingTransactions() > 0) { - txnBatch.beginNextTransaction(); - txnBatch.write(data.getBytes()); - txnBatch.write(data.getBytes()); - txnBatch.commit(); - } // while - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - if (txnBatch != null) { - try { - txnBatch.close(); - } catch (Exception e) { - LOG.error("txnBatch.close() failed: " + e.getMessage(), e); - conn.close(); - } - } - try { - conn.close(); - } catch (Exception e) { - LOG.error("conn.close() failed: " + e.getMessage(), e); - } - - } - } - } - - /** - * Make sure that creating an already existing partion is handled gracefully - * @throws Exception - */ - @Test - public void testCreatePartition() throws Exception { - final HiveEndPoint ep = new HiveEndPoint(metaStoreURI, dbName, tblName, partitionVals); - StreamingConnection conn = ep.newConnection(true); - conn.close(); - conn = ep.newConnection(true); - conn.close(); - } - @Test - public void testConcurrentTransactionBatchCommits() throws Exception { - final HiveEndPoint ep = new HiveEndPoint(metaStoreURI, dbName, tblName, partitionVals); - List writers = new ArrayList(3); - writers.add(new WriterThd(ep, "1,Matrix")); - writers.add(new WriterThd(ep, "2,Gandhi")); - writers.add(new WriterThd(ep, "3,Silence")); - - for(WriterThd w : writers) { - w.start(); - } - for(WriterThd w : writers) { - w.join(); - } - for(WriterThd w : writers) { - if(w.error != null) { - Assert.assertFalse("Writer thread" + w.getName() + " died: " + w.error.getMessage() + - " See log file for stack trace", true); - } - } - } - - - private ArrayList dumpBucket(Path orcFile) throws IOException { - org.apache.hadoop.fs.FileSystem fs = org.apache.hadoop.fs.FileSystem.getLocal(new Configuration()); - Reader reader = OrcFile.createReader(orcFile, - OrcFile.readerOptions(conf).filesystem(fs)); - - RecordReader rows = reader.rows(); - StructObjectInspector inspector = (StructObjectInspector) reader - .getObjectInspector(); - - System.out.format("Found Bucket File : %s \n", orcFile.getName()); - ArrayList result = new ArrayList(); - while (rows.hasNext()) { - Object row = rows.next(null); - SampleRec rec = (SampleRec) deserializeDeltaFileRow(row, inspector)[5]; - result.add(rec); - } - - return result; - } - - // Assumes stored data schema = [acid fields],string,int,string - // return array of 6 fields, where the last field has the actual data - private static Object[] deserializeDeltaFileRow(Object row, StructObjectInspector inspector) { - List fields = inspector.getAllStructFieldRefs(); - - WritableIntObjectInspector f0ins = (WritableIntObjectInspector) fields.get(0).getFieldObjectInspector(); - WritableLongObjectInspector f1ins = (WritableLongObjectInspector) fields.get(1).getFieldObjectInspector(); - WritableIntObjectInspector f2ins = (WritableIntObjectInspector) fields.get(2).getFieldObjectInspector(); - WritableLongObjectInspector f3ins = (WritableLongObjectInspector) fields.get(3).getFieldObjectInspector(); - WritableLongObjectInspector f4ins = (WritableLongObjectInspector) fields.get(4).getFieldObjectInspector(); - StructObjectInspector f5ins = (StructObjectInspector) fields.get(5).getFieldObjectInspector(); - - int f0 = f0ins.get(inspector.getStructFieldData(row, fields.get(0))); - long f1 = f1ins.get(inspector.getStructFieldData(row, fields.get(1))); - int f2 = f2ins.get(inspector.getStructFieldData(row, fields.get(2))); - long f3 = f3ins.get(inspector.getStructFieldData(row, fields.get(3))); - long f4 = f4ins.get(inspector.getStructFieldData(row, fields.get(4))); - SampleRec f5 = deserializeInner(inspector.getStructFieldData(row, fields.get(5)), f5ins); - - return new Object[] {f0, f1, f2, f3, f4, f5}; - } - - // Assumes row schema => string,int,string - private static SampleRec deserializeInner(Object row, StructObjectInspector inspector) { - List fields = inspector.getAllStructFieldRefs(); - - WritableStringObjectInspector f0ins = (WritableStringObjectInspector) fields.get(0).getFieldObjectInspector(); - WritableIntObjectInspector f1ins = (WritableIntObjectInspector) fields.get(1).getFieldObjectInspector(); - WritableStringObjectInspector f2ins = (WritableStringObjectInspector) fields.get(2).getFieldObjectInspector(); - - String f0 = f0ins.getPrimitiveJavaObject(inspector.getStructFieldData(row, fields.get(0))); - int f1 = f1ins.get(inspector.getStructFieldData(row, fields.get(1))); - String f2 = f2ins.getPrimitiveJavaObject(inspector.getStructFieldData(row, fields.get(2))); - return new SampleRec(f0, f1, f2); - } - - @Test - public void testBucketing() throws Exception { - String agentInfo = "UT_" + Thread.currentThread().getName(); - dropDB(msClient, dbName3); - dropDB(msClient, dbName4); - - // 1) Create two bucketed tables - String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db"; - dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - String[] colNames = "key1,key2,data".split(","); - String[] colTypes = "string,int,string".split(","); - String[] bucketNames = "key1,key2".split(","); - int bucketCount = 4; - createDbAndTable(driver, dbName3, tblName3, null, colNames, colTypes, bucketNames - , null, dbLocation, bucketCount); - - String dbLocation2 = dbFolder.newFolder(dbName4).getCanonicalPath() + ".db"; - dbLocation2 = dbLocation2.replaceAll("\\\\","/"); // for windows paths - String[] colNames2 = "key3,key4,data2".split(","); - String[] colTypes2 = "string,int,string".split(","); - String[] bucketNames2 = "key3,key4".split(","); - createDbAndTable(driver, dbName4, tblName4, null, colNames2, colTypes2, bucketNames2 - , null, dbLocation2, bucketCount); - - - // 2) Insert data into both tables - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName3, tblName3, null); - StreamingConnection connection = endPt.newConnection(false, agentInfo); - DelimitedInputWriter writer = new DelimitedInputWriter(colNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name0,1,Hello streaming".getBytes()); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - txnBatch.write("name4,2,more Streaming unlimited".getBytes()); - txnBatch.write("name5,2,even more Streaming unlimited".getBytes()); - txnBatch.commit(); - - - HiveEndPoint endPt2 = new HiveEndPoint(metaStoreURI, dbName4, tblName4, null); - StreamingConnection connection2 = endPt2.newConnection(false, agentInfo); - DelimitedInputWriter writer2 = new DelimitedInputWriter(colNames2,",", endPt2, connection); - TransactionBatch txnBatch2 = connection2.fetchTransactionBatch(2, writer2); - txnBatch2.beginNextTransaction(); - - txnBatch2.write("name5,2,fact3".getBytes()); // bucket 0 - txnBatch2.write("name8,2,fact3".getBytes()); // bucket 1 - txnBatch2.write("name0,1,fact1".getBytes()); // bucket 2 - - txnBatch2.commit(); - - // 3 Check data distribution in buckets - - HashMap> actual1 = dumpAllBuckets(dbLocation, tblName3); - HashMap> actual2 = dumpAllBuckets(dbLocation2, tblName4); - System.err.println("\n Table 1"); - System.err.println(actual1); - System.err.println("\n Table 2"); - System.err.println(actual2); - - // assert bucket listing is as expected - Assert.assertEquals("number of buckets does not match expectation", actual1.values().size(), 3); - Assert.assertTrue("bucket 0 shouldn't have been created", actual1.get(0) == null); - Assert.assertEquals("records in bucket does not match expectation", actual1.get(1).size(), 1); - Assert.assertEquals("records in bucket does not match expectation", actual1.get(2).size(), 2); - Assert.assertEquals("records in bucket does not match expectation", actual1.get(3).size(), 1); - } - private void runCmdOnDriver(String cmd) throws QueryFailedException { - boolean t = runDDL(driver, cmd); - Assert.assertTrue(cmd + " failed", t); - } - - - @Test - public void testFileDump() throws Exception { - String agentInfo = "UT_" + Thread.currentThread().getName(); - dropDB(msClient, dbName3); - dropDB(msClient, dbName4); - - // 1) Create two bucketed tables - String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db"; - dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - String[] colNames = "key1,key2,data".split(","); - String[] colTypes = "string,int,string".split(","); - String[] bucketNames = "key1,key2".split(","); - int bucketCount = 4; - createDbAndTable(driver, dbName3, tblName3, null, colNames, colTypes, bucketNames - , null, dbLocation, bucketCount); - - String dbLocation2 = dbFolder.newFolder(dbName4).getCanonicalPath() + ".db"; - dbLocation2 = dbLocation2.replaceAll("\\\\","/"); // for windows paths - String[] colNames2 = "key3,key4,data2".split(","); - String[] colTypes2 = "string,int,string".split(","); - String[] bucketNames2 = "key3,key4".split(","); - createDbAndTable(driver, dbName4, tblName4, null, colNames2, colTypes2, bucketNames2 - , null, dbLocation2, bucketCount); - - - // 2) Insert data into both tables - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName3, tblName3, null); - StreamingConnection connection = endPt.newConnection(false, agentInfo); - DelimitedInputWriter writer = new DelimitedInputWriter(colNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name0,1,Hello streaming".getBytes()); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - txnBatch.write("name4,2,more Streaming unlimited".getBytes()); - txnBatch.write("name5,2,even more Streaming unlimited".getBytes()); - txnBatch.commit(); - - PrintStream origErr = System.err; - ByteArrayOutputStream myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.err.flush(); - System.setErr(origErr); - - String errDump = new String(myErr.toByteArray()); - Assert.assertEquals(false, errDump.contains("file(s) are corrupted")); - // since this test runs on local file system which does not have an API to tell if files or - // open or not, we are testing for negative case even though the bucket files are still open - // for writes (transaction batch not closed yet) - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - HiveEndPoint endPt2 = new HiveEndPoint(metaStoreURI, dbName4, tblName4, null); - DelimitedInputWriter writer2 = new DelimitedInputWriter(colNames2,",", endPt2); - StreamingConnection connection2 = endPt2.newConnection(false, agentInfo); - TransactionBatch txnBatch2 = connection2.fetchTransactionBatch(2, writer2); - txnBatch2.beginNextTransaction(); - - txnBatch2.write("name5,2,fact3".getBytes()); // bucket 0 - txnBatch2.write("name8,2,fact3".getBytes()); // bucket 1 - txnBatch2.write("name0,1,fact1".getBytes()); // bucket 2 - // no data for bucket 3 -- expect 0 length bucket file - - txnBatch2.commit(); - - origErr = System.err; - myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.out.flush(); - System.err.flush(); - System.setErr(origErr); - - errDump = new String(myErr.toByteArray()); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(false, errDump.contains("file(s) are corrupted")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - } - - @Test - public void testFileDumpCorruptDataFiles() throws Exception { - dropDB(msClient, dbName3); - - // 1) Create two bucketed tables - String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db"; - dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - String[] colNames = "key1,key2,data".split(","); - String[] colTypes = "string,int,string".split(","); - String[] bucketNames = "key1,key2".split(","); - int bucketCount = 4; - createDbAndTable(driver, dbName3, tblName3, null, colNames, colTypes, bucketNames - , null, dbLocation, bucketCount); - - // 2) Insert data into both tables - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName3, tblName3, null); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(colNames,",", endPt, connection); - - // we need side file for this test, so we create 2 txn batch and test with only one - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name0,1,Hello streaming".getBytes()); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - txnBatch.write("name4,2,more Streaming unlimited".getBytes()); - txnBatch.write("name5,2,even more Streaming unlimited".getBytes()); - txnBatch.commit(); - - // intentionally corrupt some files - Path path = new Path(dbLocation); - Collection files = FileDump.getAllFilesInPath(path, conf); - int readableFooter = -1; - for (String file : files) { - if (file.contains("bucket_00000")) { - // empty out the file - corruptDataFile(file, conf, Integer.MIN_VALUE); - } else if (file.contains("bucket_00001")) { - corruptDataFile(file, conf, -1); - } else if (file.contains("bucket_00002")) { - corruptDataFile(file, conf, 100); - } else if (file.contains("bucket_00003")) { - corruptDataFile(file, conf, 100); - } - } - - PrintStream origErr = System.err; - ByteArrayOutputStream myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.err.flush(); - System.setErr(origErr); - - String errDump = new String(myErr.toByteArray()); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(true, errDump.contains("3 file(s) are corrupted")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - origErr = System.err; - myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation, "--recover", "--skip-dump"}); - System.err.flush(); - System.setErr(origErr); - - errDump = new String(myErr.toByteArray()); - Assert.assertEquals(true, errDump.contains("bucket_00001 recovered successfully!")); - Assert.assertEquals(true, errDump.contains("No readable footers found. Creating empty orc file.")); - Assert.assertEquals(true, errDump.contains("bucket_00002 recovered successfully!")); - Assert.assertEquals(true, errDump.contains("bucket_00003 recovered successfully!")); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - // test after recovery - origErr = System.err; - myErr = new ByteArrayOutputStream(); - - // replace stdout and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.err.flush(); - System.setErr(origErr); - - errDump = new String(myErr.toByteArray()); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(false, errDump.contains("file(s) are corrupted")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - // after recovery there shouldn't be any *_flush_length files - files = FileDump.getAllFilesInPath(path, conf); - for (String file : files) { - Assert.assertEquals(false, file.contains("_flush_length")); - } - - txnBatch.close(); - } - - private void corruptDataFile(final String file, final Configuration conf, final int addRemoveBytes) - throws Exception { - Path bPath = new Path(file); - Path cPath = new Path(bPath.getParent(), bPath.getName() + ".corrupt"); - FileSystem fs = bPath.getFileSystem(conf); - FileStatus fileStatus = fs.getFileStatus(bPath); - int len = addRemoveBytes == Integer.MIN_VALUE ? 0 : (int) fileStatus.getLen() + addRemoveBytes; - byte[] buffer = new byte[len]; - FSDataInputStream fdis = fs.open(bPath); - fdis.readFully(0, buffer, 0, (int) Math.min(fileStatus.getLen(), buffer.length)); - fdis.close(); - FSDataOutputStream fdos = fs.create(cPath, true); - fdos.write(buffer, 0, buffer.length); - fdos.close(); - fs.delete(bPath, false); - fs.rename(cPath, bPath); - } - - @Test - public void testFileDumpCorruptSideFiles() throws Exception { - dropDB(msClient, dbName3); - - // 1) Create two bucketed tables - String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db"; - dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths - String[] colNames = "key1,key2,data".split(","); - String[] colTypes = "string,int,string".split(","); - String[] bucketNames = "key1,key2".split(","); - int bucketCount = 4; - createDbAndTable(driver, dbName3, tblName3, null, colNames, colTypes, bucketNames - , null, dbLocation, bucketCount); - - // 2) Insert data into both tables - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, dbName3, tblName3, null); - StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName()); - DelimitedInputWriter writer = new DelimitedInputWriter(colNames,",", endPt, connection); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name0,1,Hello streaming".getBytes()); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - txnBatch.write("name4,2,more Streaming unlimited".getBytes()); - txnBatch.write("name5,2,even more Streaming unlimited".getBytes()); - txnBatch.write("name6,3,aHello streaming".getBytes()); - txnBatch.commit(); - - Map> offsetMap = new HashMap>(); - recordOffsets(conf, dbLocation, offsetMap); - - txnBatch.beginNextTransaction(); - txnBatch.write("name01,11,-Hello streaming".getBytes()); - txnBatch.write("name21,21,-Welcome to streaming".getBytes()); - txnBatch.write("name41,21,-more Streaming unlimited".getBytes()); - txnBatch.write("name51,21,-even more Streaming unlimited".getBytes()); - txnBatch.write("name02,12,--Hello streaming".getBytes()); - txnBatch.write("name22,22,--Welcome to streaming".getBytes()); - txnBatch.write("name42,22,--more Streaming unlimited".getBytes()); - txnBatch.write("name52,22,--even more Streaming unlimited".getBytes()); - txnBatch.write("name7,4,aWelcome to streaming".getBytes()); - txnBatch.write("name8,5,amore Streaming unlimited".getBytes()); - txnBatch.write("name9,6,aeven more Streaming unlimited".getBytes()); - txnBatch.write("name10,7,bHello streaming".getBytes()); - txnBatch.write("name11,8,bWelcome to streaming".getBytes()); - txnBatch.write("name12,9,bmore Streaming unlimited".getBytes()); - txnBatch.write("name13,10,beven more Streaming unlimited".getBytes()); - txnBatch.commit(); - - recordOffsets(conf, dbLocation, offsetMap); - - // intentionally corrupt some files - Path path = new Path(dbLocation); - Collection files = FileDump.getAllFilesInPath(path, conf); - for (String file : files) { - if (file.contains("bucket_00000")) { - corruptSideFile(file, conf, offsetMap, "bucket_00000", -1); // corrupt last entry - } else if (file.contains("bucket_00001")) { - corruptSideFile(file, conf, offsetMap, "bucket_00001", 0); // empty out side file - } else if (file.contains("bucket_00002")) { - corruptSideFile(file, conf, offsetMap, "bucket_00002", 3); // total 3 entries (2 valid + 1 fake) - } else if (file.contains("bucket_00003")) { - corruptSideFile(file, conf, offsetMap, "bucket_00003", 10); // total 10 entries (2 valid + 8 fake) - } - } - - PrintStream origErr = System.err; - ByteArrayOutputStream myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.err.flush(); - System.setErr(origErr); - - String errDump = new String(myErr.toByteArray()); - Assert.assertEquals(true, errDump.contains("bucket_00000_flush_length [length: 11")); - Assert.assertEquals(true, errDump.contains("bucket_00001_flush_length [length: 0")); - Assert.assertEquals(true, errDump.contains("bucket_00002_flush_length [length: 24")); - Assert.assertEquals(true, errDump.contains("bucket_00003_flush_length [length: 80")); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(true, errDump.contains("4 file(s) are corrupted")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - origErr = System.err; - myErr = new ByteArrayOutputStream(); - - // replace stderr and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation, "--recover", "--skip-dump"}); - System.err.flush(); - System.setErr(origErr); - - errDump = new String(myErr.toByteArray()); - Assert.assertEquals(true, errDump.contains("bucket_00000 recovered successfully!")); - Assert.assertEquals(true, errDump.contains("bucket_00001 recovered successfully!")); - Assert.assertEquals(true, errDump.contains("bucket_00002 recovered successfully!")); - Assert.assertEquals(true, errDump.contains("bucket_00003 recovered successfully!")); - List offsets = offsetMap.get("bucket_00000"); - Assert.assertEquals(true, errDump.contains("Readable footerOffsets: " + offsets.toString())); - offsets = offsetMap.get("bucket_00001"); - Assert.assertEquals(true, errDump.contains("Readable footerOffsets: " + offsets.toString())); - offsets = offsetMap.get("bucket_00002"); - Assert.assertEquals(true, errDump.contains("Readable footerOffsets: " + offsets.toString())); - offsets = offsetMap.get("bucket_00003"); - Assert.assertEquals(true, errDump.contains("Readable footerOffsets: " + offsets.toString())); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - // test after recovery - origErr = System.err; - myErr = new ByteArrayOutputStream(); - - // replace stdout and run command - System.setErr(new PrintStream(myErr)); - FileDump.main(new String[]{dbLocation}); - System.err.flush(); - System.setErr(origErr); - - errDump = new String(myErr.toByteArray()); - Assert.assertEquals(false, errDump.contains("Exception")); - Assert.assertEquals(false, errDump.contains("file(s) are corrupted")); - Assert.assertEquals(false, errDump.contains("is still open for writes.")); - - // after recovery there shouldn't be any *_flush_length files - files = FileDump.getAllFilesInPath(path, conf); - for (String file : files) { - Assert.assertEquals(false, file.contains("_flush_length")); - } - - txnBatch.close(); - } - - private void corruptSideFile(final String file, final HiveConf conf, - final Map> offsetMap, final String key, final int numEntries) - throws IOException { - Path dataPath = new Path(file); - Path sideFilePath = OrcAcidUtils.getSideFile(dataPath); - Path cPath = new Path(sideFilePath.getParent(), sideFilePath.getName() + ".corrupt"); - FileSystem fs = sideFilePath.getFileSystem(conf); - List offsets = offsetMap.get(key); - long lastOffset = offsets.get(offsets.size() - 1); - FSDataOutputStream fdos = fs.create(cPath, true); - // corrupt last entry - if (numEntries < 0) { - byte[] lastOffsetBytes = longToBytes(lastOffset); - for (int i = 0; i < offsets.size() - 1; i++) { - fdos.writeLong(offsets.get(i)); - } - - fdos.write(lastOffsetBytes, 0, 3); - } else if (numEntries > 0) { - int firstRun = Math.min(offsets.size(), numEntries); - // add original entries - for (int i=0; i < firstRun; i++) { - fdos.writeLong(offsets.get(i)); - } - - // add fake entries - int remaining = numEntries - firstRun; - for (int i = 0; i < remaining; i++) { - fdos.writeLong(lastOffset + ((i + 1) * 100)); - } - } - - fdos.close(); - fs.delete(sideFilePath, false); - fs.rename(cPath, sideFilePath); - } - - private byte[] longToBytes(long x) { - ByteBuffer buffer = ByteBuffer.allocate(8); - buffer.putLong(x); - return buffer.array(); - } - - private void recordOffsets(final HiveConf conf, final String dbLocation, - final Map> offsetMap) throws IOException { - Path path = new Path(dbLocation); - Collection files = FileDump.getAllFilesInPath(path, conf); - for (String file: files) { - Path bPath = new Path(file); - FileSystem fs = bPath.getFileSystem(conf); - FileStatus fileStatus = fs.getFileStatus(bPath); - long len = fileStatus.getLen(); - - if (file.contains("bucket_00000")) { - if (offsetMap.containsKey("bucket_00000")) { - List offsets = offsetMap.get("bucket_00000"); - offsets.add(len); - offsetMap.put("bucket_00000", offsets); - } else { - List offsets = new ArrayList(); - offsets.add(len); - offsetMap.put("bucket_00000", offsets); - } - } else if (file.contains("bucket_00001")) { - if (offsetMap.containsKey("bucket_00001")) { - List offsets = offsetMap.get("bucket_00001"); - offsets.add(len); - offsetMap.put("bucket_00001", offsets); - } else { - List offsets = new ArrayList(); - offsets.add(len); - offsetMap.put("bucket_00001", offsets); - } - } else if (file.contains("bucket_00002")) { - if (offsetMap.containsKey("bucket_00002")) { - List offsets = offsetMap.get("bucket_00002"); - offsets.add(len); - offsetMap.put("bucket_00002", offsets); - } else { - List offsets = new ArrayList(); - offsets.add(len); - offsetMap.put("bucket_00002", offsets); - } - } else if (file.contains("bucket_00003")) { - if (offsetMap.containsKey("bucket_00003")) { - List offsets = offsetMap.get("bucket_00003"); - offsets.add(len); - offsetMap.put("bucket_00003", offsets); - } else { - List offsets = new ArrayList(); - offsets.add(len); - offsetMap.put("bucket_00003", offsets); - } - } - } - } - - @Test - public void testErrorHandling() throws Exception { - String agentInfo = "UT_" + Thread.currentThread().getName(); - runCmdOnDriver("create database testErrors"); - runCmdOnDriver("use testErrors"); - runCmdOnDriver("create table T(a int, b int) clustered by (b) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')"); - - HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testErrors", "T", null); - StreamingConnection connection = endPt.newConnection(false, agentInfo); - DelimitedInputWriter innerWriter = new DelimitedInputWriter("a,b".split(","),",", endPt, connection); - FaultyWriter writer = new FaultyWriter(innerWriter); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.close(); - txnBatch.heartbeat();//this is no-op on closed batch - txnBatch.abort();//ditto - GetOpenTxnsInfoResponse r = msClient.showTxns(); - Assert.assertEquals("HWM didn't match", 17, r.getTxn_high_water_mark()); - List ti = r.getOpen_txns(); - Assert.assertEquals("wrong status ti(0)", TxnState.ABORTED, ti.get(0).getState()); - Assert.assertEquals("wrong status ti(1)", TxnState.ABORTED, ti.get(1).getState()); - - Exception expectedEx = null; - try { - txnBatch.beginNextTransaction(); - } - catch(IllegalStateException ex) { - expectedEx = ex; - } - Assert.assertTrue("beginNextTransaction() should have failed", - expectedEx != null && expectedEx.getMessage().contains("has been closed()")); - expectedEx = null; - try { - txnBatch.write("name0,1,Hello streaming".getBytes()); - } - catch(IllegalStateException ex) { - expectedEx = ex; - } - Assert.assertTrue("write() should have failed", - expectedEx != null && expectedEx.getMessage().contains("has been closed()")); - expectedEx = null; - try { - txnBatch.commit(); - } - catch(IllegalStateException ex) { - expectedEx = ex; - } - Assert.assertTrue("commit() should have failed", - expectedEx != null && expectedEx.getMessage().contains("has been closed()")); - - txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - txnBatch.write("name4,2,more Streaming unlimited".getBytes()); - txnBatch.write("name5,2,even more Streaming unlimited".getBytes()); - txnBatch.commit(); - - //test toString() - String s = txnBatch.toString(); - Assert.assertTrue("Actual: " + s, s.contains("LastUsed " + JavaUtils.txnIdToString(txnBatch.getCurrentTxnId()))); - Assert.assertTrue("Actual: " + s, s.contains("TxnStatus[CO]")); - - expectedEx = null; - txnBatch.beginNextTransaction(); - writer.enableErrors(); - try { - txnBatch.write("name6,2,Doh!".getBytes()); - } - catch(StreamingIOFailure ex) { - expectedEx = ex; - txnBatch.getCurrentTransactionState(); - txnBatch.getCurrentTxnId();//test it doesn't throw ArrayIndexOutOfBounds... - } - Assert.assertTrue("Wrong exception: " + (expectedEx != null ? expectedEx.getMessage() : "?"), - expectedEx != null && expectedEx.getMessage().contains("Simulated fault occurred")); - expectedEx = null; - try { - txnBatch.commit(); - } - catch(IllegalStateException ex) { - expectedEx = ex; - } - Assert.assertTrue("commit() should have failed", - expectedEx != null && expectedEx.getMessage().contains("has been closed()")); - - //test toString() - s = txnBatch.toString(); - Assert.assertTrue("Actual: " + s, s.contains("LastUsed " + JavaUtils.txnIdToString(txnBatch.getCurrentTxnId()))); - Assert.assertTrue("Actual: " + s, s.contains("TxnStatus[CA]")); - - r = msClient.showTxns(); - Assert.assertEquals("HWM didn't match", 19, r.getTxn_high_water_mark()); - ti = r.getOpen_txns(); - Assert.assertEquals("wrong status ti(0)", TxnState.ABORTED, ti.get(0).getState()); - Assert.assertEquals("wrong status ti(1)", TxnState.ABORTED, ti.get(1).getState()); - //txnid 3 was committed and thus not open - Assert.assertEquals("wrong status ti(2)", TxnState.ABORTED, ti.get(2).getState()); - - writer.disableErrors(); - txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("name2,2,Welcome to streaming".getBytes()); - writer.enableErrors(); - expectedEx = null; - try { - txnBatch.commit(); - } - catch(StreamingIOFailure ex) { - expectedEx = ex; - } - Assert.assertTrue("Wrong exception: " + (expectedEx != null ? expectedEx.getMessage() : "?"), - expectedEx != null && expectedEx.getMessage().contains("Simulated fault occurred")); - - r = msClient.showTxns(); - Assert.assertEquals("HWM didn't match", 21, r.getTxn_high_water_mark()); - ti = r.getOpen_txns(); - Assert.assertEquals("wrong status ti(3)", TxnState.ABORTED, ti.get(3).getState()); - Assert.assertEquals("wrong status ti(4)", TxnState.ABORTED, ti.get(4).getState()); - - txnBatch.abort(); - } - - // assumes un partitioned table - // returns a map > - private HashMap> dumpAllBuckets(String dbLocation, String tableName) - throws IOException { - HashMap> result = new HashMap>(); - - for (File deltaDir : new File(dbLocation + "/" + tableName).listFiles()) { - if(!deltaDir.getName().startsWith("delta")) { - continue; - } - File[] bucketFiles = deltaDir.listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - String name = pathname.getName(); - return !name.startsWith("_") && !name.startsWith("."); - } - }); - for (File bucketFile : bucketFiles) { - if(bucketFile.toString().endsWith("length")) { - continue; - } - Integer bucketNum = getBucketNumber(bucketFile); - ArrayList recs = dumpBucket(new Path(bucketFile.toString())); - result.put(bucketNum, recs); - } - } - return result; - } - - //assumes bucket_NNNNN format of file name - private Integer getBucketNumber(File bucketFile) { - String fname = bucketFile.getName(); - int start = fname.indexOf('_'); - String number = fname.substring(start+1, fname.length()); - return Integer.parseInt(number); - } - - // delete db and all tables in it - public static void dropDB(IMetaStoreClient client, String databaseName) { - try { - for (String table : client.listTableNamesByFilter(databaseName, "", (short)-1)) { - client.dropTable(databaseName, table, true, true); - } - client.dropDatabase(databaseName); - } catch (TException e) { - } - - } - - - - ///////// -------- UTILS ------- ///////// - // returns Path of the partition created (if any) else Path of table - private static Path createDbAndTable(IDriver driver, String databaseName, - String tableName, List partVals, - String[] colNames, String[] colTypes, - String[] bucketCols, - String[] partNames, String dbLocation, int bucketCount) - throws Exception { - - String dbUri = "raw://" + new Path(dbLocation).toUri().toString(); - String tableLoc = dbUri + Path.SEPARATOR + tableName; - - runDDL(driver, "create database IF NOT EXISTS " + databaseName + " location '" + dbUri + "'"); - runDDL(driver, "use " + databaseName); - String crtTbl = "create table " + tableName + - " ( " + getTableColumnsStr(colNames,colTypes) + " )" + - getPartitionStmtStr(partNames) + - " clustered by ( " + join(bucketCols, ",") + " )" + - " into " + bucketCount + " buckets " + - " stored as orc " + - " location '" + tableLoc + "'" + - " TBLPROPERTIES ('transactional'='true') "; - runDDL(driver, crtTbl); - if(partNames!=null && partNames.length!=0) { - return addPartition(driver, tableName, partVals, partNames); - } - return new Path(tableLoc); - } - - private static Path addPartition(IDriver driver, String tableName, List partVals, String[] partNames) - throws Exception { - String partSpec = getPartsSpec(partNames, partVals); - String addPart = "alter table " + tableName + " add partition ( " + partSpec + " )"; - runDDL(driver, addPart); - return getPartitionPath(driver, tableName, partSpec); - } - - private static Path getPartitionPath(IDriver driver, String tableName, String partSpec) throws Exception { - ArrayList res = queryTable(driver, "describe extended " + tableName + " PARTITION (" + partSpec + ")"); - String partInfo = res.get(res.size() - 1); - int start = partInfo.indexOf("location:") + "location:".length(); - int end = partInfo.indexOf(",",start); - return new Path( partInfo.substring(start,end) ); - } - - private static String getTableColumnsStr(String[] colNames, String[] colTypes) { - StringBuilder sb = new StringBuilder(); - for (int i=0; i < colNames.length; ++i) { - sb.append(colNames[i]).append(" ").append(colTypes[i]); - if (i partVals) { - StringBuilder sb = new StringBuilder(); - for (int i=0; i < partVals.size(); ++i) { - sb.append(partNames[i]).append(" = '").append(partVals.get(i)).append("'"); - if(i < partVals.size()-1) { - sb.append(","); - } - } - return sb.toString(); - } - - private static String join(String[] values, String delimiter) { - if(values==null) { - return null; - } - StringBuilder strbuf = new StringBuilder(); - - boolean first = true; - - for (Object value : values) { - if (!first) { strbuf.append(delimiter); } else { first = false; } - strbuf.append(value.toString()); - } - - return strbuf.toString(); - } - private static String getPartitionStmtStr(String[] partNames) { - if ( partNames == null || partNames.length == 0) { - return ""; - } - return " partitioned by (" + getTablePartsStr(partNames) + " )"; - } - - private static boolean runDDL(IDriver driver, String sql) throws QueryFailedException { - LOG.debug(sql); - System.out.println(sql); - //LOG.debug("Running Hive Query: "+ sql); - try { - driver.run(sql); - return true; - } catch (CommandProcessorException e) { - LOG.error("Statement: " + sql + " failed: " + e); - return false; - } - } - - - private static ArrayList queryTable(IDriver driver, String query) throws IOException { - try { - driver.run(query); - } catch (CommandProcessorException e) { - throw new RuntimeException(query + " failed: " + e); - } - ArrayList res = new ArrayList(); - driver.getResults(res); - return res; - } - - private static class SampleRec { - public String field1; - public int field2; - public String field3; - - public SampleRec(String field1, int field2, String field3) { - this.field1 = field1; - this.field2 = field2; - this.field3 = field3; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SampleRec that = (SampleRec) o; - - if (field2 != that.field2) { - return false; - } - if (field1 != null ? !field1.equals(that.field1) : that.field1 != null) { - return false; - } - return !(field3 != null ? !field3.equals(that.field3) : that.field3 != null); - - } - - @Override - public int hashCode() { - int result = field1 != null ? field1.hashCode() : 0; - result = 31 * result + field2; - result = 31 * result + (field3 != null ? field3.hashCode() : 0); - return result; - } - - @Override - public String toString() { - return " { " + - "'" + field1 + '\'' + - "," + field2 + - ",'" + field3 + '\'' + - " }"; - } - } - /** - * This is test-only wrapper around the real RecordWriter. - * It can simulate faults from lower levels to test error handling logic. - */ - private static final class FaultyWriter implements RecordWriter { - private final RecordWriter delegate; - private boolean shouldThrow = false; - - private FaultyWriter(RecordWriter delegate) { - assert delegate != null; - this.delegate = delegate; - } - @Override - public void write(long writeId, byte[] record) throws StreamingException { - delegate.write(writeId, record); - produceFault(); - } - @Override - public void flush() throws StreamingException { - delegate.flush(); - produceFault(); - } - @Override - public void clear() throws StreamingException { - delegate.clear(); - } - @Override - public void newBatch(Long minTxnId, Long maxTxnID) throws StreamingException { - delegate.newBatch(minTxnId, maxTxnID); - } - @Override - public void closeBatch() throws StreamingException { - delegate.closeBatch(); - } - - /** - * allows testing of "unexpected" errors - * @throws StreamingIOFailure - */ - private void produceFault() throws StreamingIOFailure { - if(shouldThrow) { - throw new StreamingIOFailure("Simulated fault occurred"); - } - } - void enableErrors() { - shouldThrow = true; - } - void disableErrors() { - shouldThrow = false; - } - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java deleted file mode 100644 index d38950e5fd..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate; - -import java.util.List; - -import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClient; -import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClientBuilder; -import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable; -import org.apache.hive.hcatalog.streaming.mutate.client.Transaction; -import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinator; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinatorBuilder; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory; - -public class ExampleUseCase { - - private String metaStoreUri; - private String databaseName; - private String tableName; - private boolean createPartitions = true; - private List partitionValues1, partitionValues2, partitionValues3; - private Object record1, record2, record3; - private MutatorFactory mutatorFactory; - - /* This is an illustration, not a functioning example. */ - public void example() throws Exception { - // CLIENT/TOOL END - // - // Singleton instance in the job client - - // Create a client to manage our transaction - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(databaseName, tableName, createPartitions) - .metaStoreUri(metaStoreUri) - .build(); - - // Get the transaction - Transaction transaction = client.newTransaction(); - - // Get serializable details of the destination tables - List tables = client.getTables(); - - transaction.begin(); - - // CLUSTER / WORKER END - // - // Job submitted to the cluster - // - - BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(tables.get(0).getTotalBuckets()); - record1 = bucketIdResolver.attachBucketIdToRecord(record1); - - // -------------------------------------------------------------- - // DATA SHOULD GET SORTED BY YOUR ETL/MERGE PROCESS HERE - // - // Group the data by (partitionValues, ROW__ID.bucketId) - // Order the groups by (ROW__ID.writeId, ROW__ID.rowId) - // -------------------------------------------------------------- - - // One of these runs at the output of each reducer - // - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(tables.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - coordinator.insert(partitionValues1, record1); - coordinator.update(partitionValues2, record2); - coordinator.delete(partitionValues3, record3); - - coordinator.close(); - - // CLIENT/TOOL END - // - // The tasks have completed, control is back at the tool - - transaction.commit(); - - client.close(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java deleted file mode 100644 index 365d20c80f..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming.mutate; - -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.io.Text; - -public class MutableRecord { - - // Column 0 - public final int id; - // Column 1 - public final Text msg; - // Column 2 - public RecordIdentifier rowId; - - public MutableRecord(int id, String msg, RecordIdentifier rowId) { - this.id = id; - this.msg = new Text(msg); - this.rowId = rowId; - } - - public MutableRecord(int id, String msg) { - this.id = id; - this.msg = new Text(msg); - rowId = null; - } - - @Override - public String toString() { - return "MutableRecord [id=" + id + ", msg=" + msg + ", rowId=" + rowId + "]"; - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java deleted file mode 100644 index c05ddcf983..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver; -import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolverImpl; -import org.apache.hive.hcatalog.streaming.mutate.worker.Mutator; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorImpl; -import org.apache.hive.hcatalog.streaming.mutate.worker.RecordInspector; -import org.apache.hive.hcatalog.streaming.mutate.worker.RecordInspectorImpl; - -public class ReflectiveMutatorFactory implements MutatorFactory { - - private final int recordIdColumn; - private final ObjectInspector objectInspector; - private final Configuration configuration; - private final int[] bucketColumnIndexes; - - public ReflectiveMutatorFactory(Configuration configuration, Class recordClass, int recordIdColumn, - int[] bucketColumnIndexes) { - this.configuration = configuration; - this.recordIdColumn = recordIdColumn; - this.bucketColumnIndexes = bucketColumnIndexes; - objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(recordClass, - ObjectInspectorFactory.ObjectInspectorOptions.JAVA); - } - - @Override - public Mutator newMutator(AcidOutputFormat outputFormat, long writeId, Path partitionPath, int bucketId) - throws IOException { - return new MutatorImpl(configuration, recordIdColumn, objectInspector, outputFormat, writeId, partitionPath, - bucketId); - } - - @Override - public RecordInspector newRecordInspector() { - return new RecordInspectorImpl(objectInspector, recordIdColumn); - } - - @Override - public BucketIdResolver newBucketIdResolver(int totalBuckets) { - return new BucketIdResolverImpl(objectInspector, recordIdColumn, totalBuckets, bucketColumnIndexes); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java deleted file mode 100644 index 86f762e97c..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java +++ /dev/null @@ -1,223 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming.mutate; - -import static org.junit.Assert.assertEquals; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.TableName; -import org.apache.hadoop.hive.common.ValidTxnList; -import org.apache.hadoop.hive.common.ValidWriteIdList; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TableValidWriteIds; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils; -import org.apache.hadoop.hive.ql.io.AcidInputFormat.AcidRecordReader; -import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.IOConstants; -import org.apache.hadoop.hive.ql.io.AcidUtils.Directory; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; -import org.apache.thrift.TException; - -public class StreamingAssert { - - public static class Factory { - private IMetaStoreClient metaStoreClient; - private final HiveConf conf; - - public Factory(IMetaStoreClient metaStoreClient, HiveConf conf) { - this.metaStoreClient = metaStoreClient; - this.conf = conf; - } - - public StreamingAssert newStreamingAssert(Table table) throws Exception { - return newStreamingAssert(table, Collections. emptyList()); - } - - public StreamingAssert newStreamingAssert(Table table, List partition) throws Exception { - return new StreamingAssert(metaStoreClient, conf, table, partition); - } - } - - private Table table; - private List partition; - private IMetaStoreClient metaStoreClient; - private Directory dir; - private ValidWriteIdList writeIds; - private ValidTxnList validTxnList; - private List currentDeltas; - private long min; - private long max; - private Path partitionLocation; - - StreamingAssert(IMetaStoreClient metaStoreClient, HiveConf conf, Table table, List partition) - throws Exception { - this.metaStoreClient = metaStoreClient; - this.table = table; - this.partition = partition; - - - validTxnList = metaStoreClient.getValidTxns(); - conf.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); - List v = metaStoreClient.getValidWriteIds(Collections - .singletonList(TableName.getDbTable(table.getDbName(), table.getTableName())), validTxnList.writeToString()); - writeIds = TxnCommonUtils.createValidReaderWriteIdList(v.get(0)); - - partitionLocation = getPartitionLocation(); - dir = AcidUtils.getAcidState(null, partitionLocation, conf, writeIds, null, false, null, true); - assertEquals(0, dir.getObsolete().size()); - assertEquals(0, dir.getOriginalFiles().size()); - - currentDeltas = dir.getCurrentDirectories(); - min = Long.MAX_VALUE; - max = Long.MIN_VALUE; - System.out.println("Files found: "); - for (AcidUtils.ParsedDelta parsedDelta : currentDeltas) { - System.out.println(parsedDelta.getPath().toString()); - max = Math.max(parsedDelta.getMaxWriteId(), max); - min = Math.min(parsedDelta.getMinWriteId(), min); - } - } - - public void assertExpectedFileCount(int expectedFileCount) { - assertEquals(expectedFileCount, currentDeltas.size()); - } - - public void assertNothingWritten() { - assertExpectedFileCount(0); - } - - public void assertMinWriteId(long expectedMinWriteId) { - if (currentDeltas.isEmpty()) { - throw new AssertionError("No data"); - } - assertEquals(expectedMinWriteId, min); - } - - public void assertMaxWriteId(long expectedMaxWriteId) { - if (currentDeltas.isEmpty()) { - throw new AssertionError("No data"); - } - assertEquals(expectedMaxWriteId, max); - } - - List readRecords() throws Exception { - return readRecords(1); - } - - /** - * TODO: this would be more flexible doing a SQL select statement rather than using InputFormat directly - * see {@link org.apache.hive.hcatalog.streaming.TestStreaming#checkDataWritten2(Path, long, long, int, String, String...)} - * @param numSplitsExpected - * @return - * @throws Exception - */ - List readRecords(int numSplitsExpected) throws Exception { - if (currentDeltas.isEmpty()) { - throw new AssertionError("No data"); - } - InputFormat inputFormat = new OrcInputFormat(); - JobConf job = new JobConf(); - job.set("mapred.input.dir", partitionLocation.toString()); - job.set(hive_metastoreConstants.BUCKET_COUNT, Integer.toString(table.getSd().getNumBuckets())); - job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS, "id,msg"); - job.set(IOConstants.SCHEMA_EVOLUTION_COLUMNS_TYPES, "bigint:string"); - AcidUtils.setAcidOperationalProperties(job, true, null); - job.setBoolean(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, true); - job.set(ValidWriteIdList.VALID_WRITEIDS_KEY, writeIds.toString()); - job.set(ValidTxnList.VALID_TXNS_KEY, validTxnList.writeToString()); - InputSplit[] splits = inputFormat.getSplits(job, 1); - assertEquals(numSplitsExpected, splits.length); - - - List records = new ArrayList<>(); - for(InputSplit is : splits) { - final AcidRecordReader recordReader = (AcidRecordReader) inputFormat - .getRecordReader(is, job, Reporter.NULL); - - NullWritable key = recordReader.createKey(); - OrcStruct value = recordReader.createValue(); - - while (recordReader.next(key, value)) { - RecordIdentifier recordIdentifier = recordReader.getRecordIdentifier(); - Record record = new Record(new RecordIdentifier(recordIdentifier.getWriteId(), - recordIdentifier.getBucketProperty(), recordIdentifier.getRowId()), value.toString()); - System.out.println(record); - records.add(record); - } - recordReader.close(); - } - return records; - } - - private Path getPartitionLocation() throws NoSuchObjectException, MetaException, TException { - Path partitionLocacation; - if (partition.isEmpty()) { - partitionLocacation = new Path(table.getSd().getLocation()); - } else { - // TODO: calculate this instead. Just because we're writing to the location doesn't mean that it'll - // always be wanted in the meta store right away. - List partitionEntries = metaStoreClient.listPartitions(table.getDbName(), table.getTableName(), - partition, (short) 1); - partitionLocacation = new Path(partitionEntries.get(0).getSd().getLocation()); - } - return partitionLocacation; - } - - public static class Record { - private RecordIdentifier recordIdentifier; - private String row; - - Record(RecordIdentifier recordIdentifier, String row) { - this.recordIdentifier = recordIdentifier; - this.row = row; - } - - public RecordIdentifier getRecordIdentifier() { - return recordIdentifier; - } - - public String getRow() { - return row; - } - - @Override - public String toString() { - return "Record [recordIdentifier=" + recordIdentifier + ", row=" + row + "]"; - } - - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java deleted file mode 100644 index afda7d5c71..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java +++ /dev/null @@ -1,288 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RawLocalFileSystem; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcSerde; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.thrift.TException; - -public class StreamingTestUtils { - - public HiveConf newHiveConf(String metaStoreUri) { - HiveConf conf = new HiveConf(this.getClass()); - conf.set("fs.raw.impl", RawFileSystem.class.getName()); - if (metaStoreUri != null) { - conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreUri); - } - conf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true); - conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); - return conf; - } - - public void prepareTransactionDatabase(HiveConf conf) throws Exception { - TxnDbUtil.setConfValues(conf); - TxnDbUtil.cleanDb(conf); - TxnDbUtil.prepDb(conf); - } - - public IMetaStoreClient newMetaStoreClient(HiveConf conf) throws Exception { - return new HiveMetaStoreClient(conf); - } - - public static class RawFileSystem extends RawLocalFileSystem { - private static final URI NAME; - static { - try { - NAME = new URI("raw:///"); - } catch (URISyntaxException se) { - throw new IllegalArgumentException("bad uri", se); - } - } - - @Override - public URI getUri() { - return NAME; - } - - @Override - public String getScheme() { - return "raw"; - } - - @Override - public FileStatus getFileStatus(Path path) throws IOException { - File file = pathToFile(path); - if (!file.exists()) { - throw new FileNotFoundException("Can't find " + path); - } - // get close enough - short mod = 0; - if (file.canRead()) { - mod |= 0444; - } - if (file.canWrite()) { - mod |= 0200; - } - if (file.canExecute()) { - mod |= 0111; - } - return new FileStatus(file.length(), file.isDirectory(), 1, 1024, file.lastModified(), file.lastModified(), - FsPermission.createImmutable(mod), "owen", "users", path); - } - } - - public static DatabaseBuilder databaseBuilder(File warehouseFolder) { - return new DatabaseBuilder(warehouseFolder); - } - - public static class DatabaseBuilder { - - private Database database; - private File warehouseFolder; - - public DatabaseBuilder(File warehouseFolder) { - this.warehouseFolder = warehouseFolder; - database = new Database(); - } - - public DatabaseBuilder name(String name) { - database.setName(name); - File databaseFolder = new File(warehouseFolder, name + ".db"); - String databaseLocation = "raw://" + databaseFolder.toURI().getPath(); - database.setLocationUri(databaseLocation); - return this; - } - - public Database dropAndCreate(IMetaStoreClient metaStoreClient) throws Exception { - if (metaStoreClient == null) { - throw new IllegalArgumentException(); - } - try { - for (String table : metaStoreClient.listTableNamesByFilter(database.getName(), "", (short) -1)) { - metaStoreClient.dropTable(database.getName(), table, true, true); - } - metaStoreClient.dropDatabase(database.getName()); - } catch (TException e) { - } - metaStoreClient.createDatabase(database); - return database; - } - - public Database build() { - return database; - } - - } - - public static TableBuilder tableBuilder(Database database) { - return new TableBuilder(database); - } - - public static class TableBuilder { - - private Table table; - private StorageDescriptor sd; - private SerDeInfo serDeInfo; - private Database database; - private List> partitions; - private List columnNames; - private List columnTypes; - private List partitionKeys; - - public TableBuilder(Database database) { - this.database = database; - partitions = new ArrayList<>(); - columnNames = new ArrayList<>(); - columnTypes = new ArrayList<>(); - partitionKeys = Collections.emptyList(); - table = new Table(); - table.setDbName(database.getName()); - table.setTableType(TableType.MANAGED_TABLE.toString()); - Map tableParams = new HashMap(); - tableParams.put("transactional", Boolean.TRUE.toString()); - table.setParameters(tableParams); - - sd = new StorageDescriptor(); - sd.setInputFormat(OrcInputFormat.class.getName()); - sd.setOutputFormat(OrcOutputFormat.class.getName()); - sd.setNumBuckets(1); - table.setSd(sd); - - serDeInfo = new SerDeInfo(); - serDeInfo.setParameters(new HashMap()); - serDeInfo.getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1"); - serDeInfo.setSerializationLib(OrcSerde.class.getName()); - sd.setSerdeInfo(serDeInfo); - } - - public TableBuilder name(String name) { - sd.setLocation(database.getLocationUri() + Path.SEPARATOR + name); - table.setTableName(name); - serDeInfo.setName(name); - return this; - } - - public TableBuilder buckets(int buckets) { - sd.setNumBuckets(buckets); - return this; - } - - public TableBuilder bucketCols(List columnNames) { - sd.setBucketCols(columnNames); - return this; - } - - public TableBuilder addColumn(String columnName, String columnType) { - columnNames.add(columnName); - columnTypes.add(columnType); - return this; - } - - public TableBuilder partitionKeys(String... partitionKeys) { - this.partitionKeys = Arrays.asList(partitionKeys); - return this; - } - - public TableBuilder addPartition(String... partitionValues) { - partitions.add(Arrays.asList(partitionValues)); - return this; - } - - public TableBuilder addPartition(List partitionValues) { - partitions.add(partitionValues); - return this; - } - - public Table create(IMetaStoreClient metaStoreClient) throws Exception { - if (metaStoreClient == null) { - throw new IllegalArgumentException(); - } - return internalCreate(metaStoreClient); - } - - public Table build() throws Exception { - return internalCreate(null); - } - - private Table internalCreate(IMetaStoreClient metaStoreClient) throws Exception { - List fields = new ArrayList(columnNames.size()); - for (int i = 0; i < columnNames.size(); i++) { - fields.add(new FieldSchema(columnNames.get(i), columnTypes.get(i), "")); - } - sd.setCols(fields); - - if (!partitionKeys.isEmpty()) { - List partitionFields = new ArrayList(); - for (String partitionKey : partitionKeys) { - partitionFields.add(new FieldSchema(partitionKey, serdeConstants.STRING_TYPE_NAME, "")); - } - table.setPartitionKeys(partitionFields); - } - if (metaStoreClient != null) { - metaStoreClient.createTable(table); - } - - for (List partitionValues : partitions) { - Partition partition = new Partition(); - partition.setDbName(database.getName()); - partition.setTableName(table.getTableName()); - StorageDescriptor partitionSd = new StorageDescriptor(table.getSd()); - partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR - + Warehouse.makePartName(table.getPartitionKeys(), partitionValues)); - partition.setSd(partitionSd); - partition.setValues(partitionValues); - - if (metaStoreClient != null) { - metaStoreClient.add_partition(partition); - } - } - return table; - } - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java deleted file mode 100644 index 3d008e67ed..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java +++ /dev/null @@ -1,566 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.hcatalog.streaming.mutate; - -import static org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState.ABORTED; -import static org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState.COMMITTED; -import static org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.databaseBuilder; -import static org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.tableBuilder; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hive.hcatalog.streaming.TestStreaming; -import org.apache.hive.hcatalog.streaming.mutate.StreamingAssert.Factory; -import org.apache.hive.hcatalog.streaming.mutate.StreamingAssert.Record; -import org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.TableBuilder; -import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClient; -import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClientBuilder; -import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable; -import org.apache.hive.hcatalog.streaming.mutate.client.Transaction; -import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinator; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinatorBuilder; -import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** - * This test is based on {@link TestStreaming} and has a similar core set of tests to ensure that basic transactional - * behaviour is as expected in the {@link RecordMutator} line. This is complemented with a set of tests related to the - * use of update and delete operations. - */ -public class TestMutations { - - private static final List EUROPE_FRANCE = Arrays.asList("Europe", "France"); - private static final List EUROPE_UK = Arrays.asList("Europe", "UK"); - private static final List ASIA_INDIA = Arrays.asList("Asia", "India"); - // id - private static final int[] BUCKET_COLUMN_INDEXES = new int[] { 0 }; - private static final int RECORD_ID_COLUMN = 2; - - @Rule - public TemporaryFolder warehouseFolder = new TemporaryFolder(); - - private StreamingTestUtils testUtils = new StreamingTestUtils(); - private HiveConf conf; - private IMetaStoreClient metaStoreClient; - private String metaStoreUri; - private Database database; - private TableBuilder partitionedTableBuilder; - private TableBuilder unpartitionedTableBuilder; - private Factory assertionFactory; - - public TestMutations() throws Exception { - conf = testUtils.newHiveConf(metaStoreUri); - testUtils.prepareTransactionDatabase(conf); - metaStoreClient = testUtils.newMetaStoreClient(conf); - assertionFactory = new StreamingAssert.Factory(metaStoreClient, conf); - } - - @Before - public void setup() throws Exception { - database = databaseBuilder(warehouseFolder.getRoot()).name("testing").dropAndCreate(metaStoreClient); - - partitionedTableBuilder = tableBuilder(database) - .name("partitioned") - .addColumn("id", "int") - .addColumn("msg", "string") - .partitionKeys("continent", "country") - .bucketCols(Collections.singletonList("string")); - - unpartitionedTableBuilder = tableBuilder(database) - .name("unpartitioned") - .addColumn("id", "int") - .addColumn("msg", "string") - .bucketCols(Collections.singletonList("string")); - } - private static int encodeBucket(int bucketId) { - return BucketCodec.V1.encode( - new AcidOutputFormat.Options(null).bucket(bucketId)); - } - - @Test - public void testTransactionBatchEmptyCommitPartitioned() throws Exception { - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - transaction.begin(); - - transaction.commit(); - assertThat(transaction.getState(), is(COMMITTED)); - client.close(); - } - - @Test - public void testTransactionBatchEmptyCommitUnpartitioned() throws Exception { - Table table = unpartitionedTableBuilder.create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), false) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - transaction.begin(); - - transaction.commit(); - assertThat(transaction.getState(), is(COMMITTED)); - client.close(); - } - - @Test - public void testTransactionBatchEmptyAbortPartitioned() throws Exception { - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - coordinator.close(); - - transaction.abort(); - assertThat(transaction.getState(), is(ABORTED)); - client.close(); - } - - @Test - public void testTransactionBatchEmptyAbortUnartitioned() throws Exception { - Table table = unpartitionedTableBuilder.create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), false) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - coordinator.close(); - - transaction.abort(); - assertThat(transaction.getState(), is(ABORTED)); - client.close(); - } - - @Test - public void testTransactionBatchCommitPartitioned() throws Exception { - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - BucketIdResolver bucketIdAppender = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord record = (MutableRecord) bucketIdAppender.attachBucketIdToRecord(new MutableRecord(1, - "Hello streaming")); - coordinator.insert(ASIA_INDIA, record); - coordinator.close(); - - transaction.commit(); - - StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA); - streamingAssertions.assertMinWriteId(1L); - streamingAssertions.assertMaxWriteId(1L); - streamingAssertions.assertExpectedFileCount(1); - - List readRecords = streamingAssertions.readRecords(); - assertThat(readRecords.size(), is(1)); - assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}")); - assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - - assertThat(transaction.getState(), is(COMMITTED)); - client.close(); - } - - @Test - public void testMulti() throws Exception { - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord asiaIndiaRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1, - "Hello streaming")); - MutableRecord europeUkRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2, - "Hello streaming")); - MutableRecord europeFranceRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(3, - "Hello streaming")); - MutableRecord europeFranceRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(4, - "Bonjour streaming")); - - coordinator.insert(ASIA_INDIA, asiaIndiaRecord1); - coordinator.insert(EUROPE_UK, europeUkRecord1); - coordinator.insert(EUROPE_FRANCE, europeFranceRecord1); - coordinator.insert(EUROPE_FRANCE, europeFranceRecord2); - coordinator.close(); - - transaction.commit(); - - // ASIA_INDIA - StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA); - streamingAssertions.assertMinWriteId(1L); - streamingAssertions.assertMaxWriteId(1L); - streamingAssertions.assertExpectedFileCount(1); - - List readRecords = streamingAssertions.readRecords(); - assertThat(readRecords.size(), is(1)); - assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}")); - assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - - // EUROPE_UK - streamingAssertions = assertionFactory.newStreamingAssert(table, EUROPE_UK); - streamingAssertions.assertMinWriteId(1L); - streamingAssertions.assertMaxWriteId(1L); - streamingAssertions.assertExpectedFileCount(1); - - readRecords = streamingAssertions.readRecords(); - assertThat(readRecords.size(), is(1)); - assertThat(readRecords.get(0).getRow(), is("{2, Hello streaming}")); - assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - - // EUROPE_FRANCE - streamingAssertions = assertionFactory.newStreamingAssert(table, EUROPE_FRANCE); - streamingAssertions.assertMinWriteId(1L); - streamingAssertions.assertMaxWriteId(1L); - streamingAssertions.assertExpectedFileCount(1); - - readRecords = streamingAssertions.readRecords(); - assertThat(readRecords.size(), is(2)); - assertThat(readRecords.get(0).getRow(), is("{3, Hello streaming}")); - assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - assertThat(readRecords.get(1).getRow(), is("{4, Bonjour streaming}")); - assertThat(readRecords.get(1).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 1L))); - - client.close(); - } - - @Test - public void testTransactionBatchCommitUnpartitioned() throws Exception { - Table table = unpartitionedTableBuilder.create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), false) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord record = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1, - "Hello streaming")); - - coordinator.insert(Collections. emptyList(), record); - coordinator.close(); - - transaction.commit(); - - StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table); - streamingAssertions.assertMinWriteId(1L); - streamingAssertions.assertMaxWriteId(1L); - streamingAssertions.assertExpectedFileCount(1); - - List readRecords = streamingAssertions.readRecords(); - assertThat(readRecords.size(), is(1)); - assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}")); - assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - - assertThat(transaction.getState(), is(COMMITTED)); - client.close(); - } - - @Test - public void testTransactionBatchAbort() throws Exception { - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction transaction = client.newTransaction(); - - List destinations = client.getTables(); - - transaction.begin(); - - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - MutatorCoordinator coordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord record1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1, - "Hello streaming")); - MutableRecord record2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2, - "Welcome to streaming")); - - coordinator.insert(ASIA_INDIA, record1); - coordinator.insert(ASIA_INDIA, record2); - coordinator.close(); - - transaction.abort(); - - assertThat(transaction.getState(), is(ABORTED)); - - client.close(); - - StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA); - streamingAssertions.assertNothingWritten(); - } - - @Test - public void testUpdatesAndDeletes() throws Exception { - // Set up some base data then stream some inserts/updates/deletes to a number of partitions - MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN, - BUCKET_COLUMN_INDEXES); - - // INSERT DATA - // - Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).addPartition(EUROPE_FRANCE).create(metaStoreClient); - - MutatorClient client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction insertTransaction = client.newTransaction(); - - List destinations = client.getTables(); - - insertTransaction.begin(); - - MutatorCoordinator insertCoordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord asiaIndiaRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1, - "Namaste streaming 1")); - MutableRecord asiaIndiaRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2, - "Namaste streaming 2")); - MutableRecord europeUkRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(3, - "Hello streaming 1")); - MutableRecord europeUkRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(4, - "Hello streaming 2")); - MutableRecord europeFranceRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(5, - "Bonjour streaming 1")); - MutableRecord europeFranceRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(6, - "Bonjour streaming 2")); - - insertCoordinator.insert(ASIA_INDIA, asiaIndiaRecord1); - insertCoordinator.insert(ASIA_INDIA, asiaIndiaRecord2); - insertCoordinator.insert(EUROPE_UK, europeUkRecord1); - insertCoordinator.insert(EUROPE_UK, europeUkRecord2); - insertCoordinator.insert(EUROPE_FRANCE, europeFranceRecord1); - insertCoordinator.insert(EUROPE_FRANCE, europeFranceRecord2); - insertCoordinator.close(); - - insertTransaction.commit(); - - assertThat(insertTransaction.getState(), is(COMMITTED)); - client.close(); - - // MUTATE DATA - // - client = new MutatorClientBuilder() - .addSinkTable(table.getDbName(), table.getTableName(), true) - .metaStoreUri(metaStoreUri) - .build(); - client.connect(); - - Transaction mutateTransaction = client.newTransaction(); - - destinations = client.getTables(); - - mutateTransaction.begin(); - - MutatorCoordinator mutateCoordinator = new MutatorCoordinatorBuilder() - .metaStoreUri(metaStoreUri) - .table(destinations.get(0)) - .mutatorFactory(mutatorFactory) - .build(); - - bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets()); - MutableRecord asiaIndiaRecord3 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(20, - "Namaste streaming 3")); - - mutateCoordinator.update(ASIA_INDIA, new MutableRecord(2, "UPDATED: Namaste streaming 2", new RecordIdentifier(1L, - encodeBucket(0), 1L))); - mutateCoordinator.insert(ASIA_INDIA, asiaIndiaRecord3); - mutateCoordinator.delete(EUROPE_UK, new MutableRecord(3, "Hello streaming 1", new RecordIdentifier(1L, - encodeBucket(0), 0L))); - mutateCoordinator.delete(EUROPE_FRANCE, - new MutableRecord(5, "Bonjour streaming 1", new RecordIdentifier(1L, - encodeBucket(0), 0L))); - mutateCoordinator.update(EUROPE_FRANCE, new MutableRecord(6, "UPDATED: Bonjour streaming 2", new RecordIdentifier( - 1L, encodeBucket(0), 1L))); - mutateCoordinator.close(); - - mutateTransaction.commit(); - - assertThat(mutateTransaction.getState(), is(COMMITTED)); - - StreamingAssert indiaAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA); - indiaAssertions.assertMinWriteId(1L); - indiaAssertions.assertMaxWriteId(2L); - List indiaRecords = indiaAssertions.readRecords(2); - assertThat(indiaRecords.size(), is(3)); - assertThat(indiaRecords.get(0).getRow(), is("{1, Namaste streaming 1}")); - assertThat(indiaRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 0L))); - assertThat(indiaRecords.get(1).getRow(), is("{2, UPDATED: Namaste streaming 2}")); - assertThat(indiaRecords.get(1).getRecordIdentifier(), is(new RecordIdentifier(2L, - encodeBucket(0), 0L)));//with split update, new version of the row is a new insert - assertThat(indiaRecords.get(2).getRow(), is("{20, Namaste streaming 3}")); - assertThat(indiaRecords.get(2).getRecordIdentifier(), is(new RecordIdentifier(2L, - encodeBucket(0), 1L))); - - StreamingAssert ukAssertions = assertionFactory.newStreamingAssert(table, EUROPE_UK); - ukAssertions.assertMinWriteId(1L); - ukAssertions.assertMaxWriteId(2L); - //1 split since mutateTransaction txn just does deletes - List ukRecords = ukAssertions.readRecords(1); - assertThat(ukRecords.size(), is(1)); - assertThat(ukRecords.get(0).getRow(), is("{4, Hello streaming 2}")); - assertThat(ukRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, - encodeBucket(0), 1L))); - - StreamingAssert franceAssertions = assertionFactory.newStreamingAssert(table, EUROPE_FRANCE); - franceAssertions.assertMinWriteId(1L); - franceAssertions.assertMaxWriteId(2L); - List franceRecords = franceAssertions.readRecords(2); - assertThat(franceRecords.size(), is(1)); - assertThat(franceRecords.get(0).getRow(), is("{6, UPDATED: Bonjour streaming 2}")); - assertThat(franceRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(2L, - encodeBucket(0), 0L)));//with split update, new version of the row is a new insert - - client.close(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java deleted file mode 100644 index 1523a10bdb..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.junit.Assert.assertThat; - -import java.io.File; - -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils; -import org.junit.Test; - -public class TestAcidTableSerializer { - - @Test - public void testSerializeDeserialize() throws Exception { - Database database = StreamingTestUtils.databaseBuilder(new File("/tmp")).name("db_1").build(); - Table table = StreamingTestUtils - .tableBuilder(database) - .name("table_1") - .addColumn("one", "string") - .addColumn("two", "integer") - .partitionKeys("partition") - .addPartition("p1") - .buckets(10) - .build(); - - AcidTable acidTable = new AcidTable("db_1", "table_1", true, TableType.SINK); - acidTable.setTable(table); - acidTable.setWriteId(42L); - - String encoded = AcidTableSerializer.encode(acidTable); - System.out.println(encoded); - AcidTable decoded = AcidTableSerializer.decode(encoded); - - assertThat(decoded.getDatabaseName(), is("db_1")); - assertThat(decoded.getTableName(), is("table_1")); - assertThat(decoded.createPartitions(), is(true)); - assertThat(decoded.getOutputFormatName(), is("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")); - assertThat(decoded.getTotalBuckets(), is(10)); - assertThat(decoded.getQualifiedName(), is("DB_1.TABLE_1")); - assertThat(decoded.getWriteId(), is(42L)); - assertThat(decoded.getTableType(), is(TableType.SINK)); - assertThat(decoded.getTable(), is(table)); - } - - @Test - public void testSerializeDeserializeNoTableNoTransaction() throws Exception { - AcidTable acidTable = new AcidTable("db_1", "table_1", true, TableType.SINK); - - String encoded = AcidTableSerializer.encode(acidTable); - AcidTable decoded = AcidTableSerializer.decode(encoded); - - assertThat(decoded.getDatabaseName(), is("db_1")); - assertThat(decoded.getTableName(), is("table_1")); - assertThat(decoded.createPartitions(), is(true)); - assertThat(decoded.getOutputFormatName(), is(nullValue())); - assertThat(decoded.getTotalBuckets(), is(0)); - assertThat(decoded.getQualifiedName(), is("DB_1.TABLE_1")); - assertThat(decoded.getWriteId(), is(0L)); - assertThat(decoded.getTableType(), is(TableType.SINK)); - assertThat(decoded.getTable(), is(nullValue())); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java deleted file mode 100644 index 4222786d80..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; -import org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener; -import org.apache.thrift.TException; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestMutatorClient { - - private static final long TRANSACTION_ID = 42L; - private static final long WRITE_ID1 = 78L; - private static final long WRITE_ID2 = 33L; - private static final String TABLE_NAME_1 = "TABLE_1"; - private static final String TABLE_NAME_2 = "TABLE_2"; - private static final String DB_NAME = "DB_1"; - private static final String USER = "user"; - private static final AcidTable TABLE_1 = new AcidTable(DB_NAME, TABLE_NAME_1, true, TableType.SINK); - private static final AcidTable TABLE_2 = new AcidTable(DB_NAME, TABLE_NAME_2, true, TableType.SINK); - - @Mock - private IMetaStoreClient mockMetaStoreClient; - @Mock - private Lock mockLock; - @Mock - private Table mockTable1, mockTable2; - @Mock - private StorageDescriptor mockSd; - @Mock - private Map mockParameters; - @Mock - private HiveConf mockConfiguration; - @Mock - private LockFailureListener mockLockFailureListener; - - private MutatorClient client; - - @Before - public void configureMocks() throws Exception { - when(mockMetaStoreClient.getTable(DB_NAME, TABLE_NAME_1)).thenReturn(mockTable1); - when(mockTable1.getDbName()).thenReturn(DB_NAME); - when(mockTable1.getTableName()).thenReturn(TABLE_NAME_1); - when(mockTable1.getSd()).thenReturn(mockSd); - when(mockTable1.getParameters()).thenReturn(mockParameters); - when(mockMetaStoreClient.getTable(DB_NAME, TABLE_NAME_2)).thenReturn(mockTable2); - when(mockTable2.getDbName()).thenReturn(DB_NAME); - when(mockTable2.getTableName()).thenReturn(TABLE_NAME_2); - when(mockTable2.getSd()).thenReturn(mockSd); - when(mockTable2.getParameters()).thenReturn(mockParameters); - when(mockSd.getNumBuckets()).thenReturn(1, 2); - when(mockSd.getOutputFormat()).thenReturn(OrcOutputFormat.class.getName()); - when(mockParameters.get("transactional")).thenReturn(Boolean.TRUE.toString()); - - when(mockMetaStoreClient.openTxn(USER)).thenReturn(TRANSACTION_ID); - when(mockMetaStoreClient.allocateTableWriteId(TRANSACTION_ID, DB_NAME, TABLE_NAME_1)).thenReturn(WRITE_ID1); - when(mockMetaStoreClient.allocateTableWriteId(TRANSACTION_ID, DB_NAME, TABLE_NAME_2)).thenReturn(WRITE_ID2); - - client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER, - Collections.singletonList(TABLE_1)); - } - - @Test - public void testCheckValidTableConnect() throws Exception { - List inTables = new ArrayList<>(); - inTables.add(TABLE_1); - inTables.add(TABLE_2); - client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER, inTables); - - client.connect(); - List outTables = client.getTables(); - - assertThat(client.isConnected(), is(true)); - assertThat(outTables.size(), is(2)); - assertThat(outTables.get(0).getDatabaseName(), is(DB_NAME)); - assertThat(outTables.get(0).getTableName(), is(TABLE_NAME_1)); - assertThat(outTables.get(0).getTotalBuckets(), is(2)); - assertThat(outTables.get(0).getOutputFormatName(), is(OrcOutputFormat.class.getName())); - assertThat(outTables.get(0).getWriteId(), is(0L)); - assertThat(outTables.get(0).getTable(), is(mockTable1)); - assertThat(outTables.get(1).getDatabaseName(), is(DB_NAME)); - assertThat(outTables.get(1).getTableName(), is(TABLE_NAME_2)); - assertThat(outTables.get(1).getTotalBuckets(), is(2)); - assertThat(outTables.get(1).getOutputFormatName(), is(OrcOutputFormat.class.getName())); - assertThat(outTables.get(1).getWriteId(), is(0L)); - assertThat(outTables.get(1).getTable(), is(mockTable2)); - } - - @Test - public void testCheckNonTransactionalTableConnect() throws Exception { - when(mockParameters.get("transactional")).thenReturn(Boolean.FALSE.toString()); - - try { - client.connect(); - fail(); - } catch (ConnectionException e) { - } - - assertThat(client.isConnected(), is(false)); - } - - @Test - public void testCheckUnBucketedTableConnect() throws Exception { - when(mockSd.getNumBuckets()).thenReturn(0); - - try { - client.connect(); - fail(); - } catch (ConnectionException e) { - } - - assertThat(client.isConnected(), is(false)); - } - - @Test - public void testMetaStoreFailsOnConnect() throws Exception { - when(mockMetaStoreClient.getTable(anyString(), anyString())).thenThrow(new TException()); - - try { - client.connect(); - fail(); - } catch (ConnectionException e) { - } - - assertThat(client.isConnected(), is(false)); - } - - @Test(expected = ConnectionException.class) - public void testGetDestinationsFailsIfNotConnected() throws Exception { - client.getTables(); - } - - @Test - public void testNewTransaction() throws Exception { - List inTables = new ArrayList<>(); - inTables.add(TABLE_1); - inTables.add(TABLE_2); - client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER, inTables); - - client.connect(); - Transaction transaction = client.newTransaction(); - List outTables = client.getTables(); - - assertThat(client.isConnected(), is(true)); - - assertThat(transaction.getTransactionId(), is(TRANSACTION_ID)); - assertThat(transaction.getState(), is(TxnState.INACTIVE)); - assertThat(outTables.get(0).getWriteId(), is(WRITE_ID1)); - assertThat(outTables.get(1).getWriteId(), is(WRITE_ID2)); - } - - @Test - public void testCloseClosesClient() throws Exception { - client.close(); - assertThat(client.isConnected(), is(false)); - verify(mockMetaStoreClient).close(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java deleted file mode 100644 index 4d30c253c6..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hive.hcatalog.streaming.TransactionBatch; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock; -import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockException; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestTransaction { - - private static final String USER = "user"; - private static final long TRANSACTION_ID = 10L; - - @Mock - private Lock mockLock; - @Mock - private IMetaStoreClient mockMetaStoreClient; - - private Transaction transaction; - - @Before - public void createTransaction() throws Exception { - when(mockLock.getUser()).thenReturn(USER); - when(mockMetaStoreClient.openTxn(USER)).thenReturn(TRANSACTION_ID); - transaction = new Transaction(mockMetaStoreClient, mockLock); - } - - @Test - public void testInitialState() { - assertThat(transaction.getState(), is(TransactionBatch.TxnState.INACTIVE)); - assertThat(transaction.getTransactionId(), is(TRANSACTION_ID)); - } - - @Test - public void testBegin() throws Exception { - transaction.begin(); - - verify(mockLock).acquire(TRANSACTION_ID); - assertThat(transaction.getState(), is(TransactionBatch.TxnState.OPEN)); - } - - @Test - public void testBeginLockFails() throws Exception { - doThrow(new LockException("")).when(mockLock).acquire(TRANSACTION_ID); - - try { - transaction.begin(); - } catch (TransactionException ignore) { - } - - assertThat(transaction.getState(), is(TransactionBatch.TxnState.INACTIVE)); - } - - @Test - public void testCommit() throws Exception { - transaction.commit(); - - verify(mockLock).release(); - verify(mockMetaStoreClient).commitTxn(TRANSACTION_ID); - assertThat(transaction.getState(), is(TransactionBatch.TxnState.COMMITTED)); - } - - @Test(expected = TransactionException.class) - public void testCommitLockFails() throws Exception { - doThrow(new LockException("")).when(mockLock).release(); - transaction.commit(); - } - - @Test - public void testAbort() throws Exception { - transaction.abort(); - - verify(mockLock).release(); - verify(mockMetaStoreClient).rollbackTxn(TRANSACTION_ID); - assertThat(transaction.getState(), is(TransactionBatch.TxnState.ABORTED)); - } - - @Test(expected = TransactionException.class) - public void testAbortLockFails() throws Exception { - doThrow(new LockException("")).when(mockLock).release(); - transaction.abort(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java deleted file mode 100644 index 33e29cf924..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.verify; - -import java.util.Arrays; -import java.util.List; - -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.NoSuchLockException; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.thrift.TException; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestHeartbeatTimerTask { - - private static final long TRANSACTION_ID = 10L; - private static final long LOCK_ID = 1L; - private static final List TABLES = createTable(); - - @Mock - private IMetaStoreClient mockMetaStoreClient; - @Mock - private LockFailureListener mockListener; - - private HeartbeatTimerTask task; - - @Before - public void create() throws Exception { - task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID); - } - - @Test - public void testRun() throws Exception { - task.run(); - - verify(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - } - - @Test - public void testRunNullTransactionId() throws Exception { - task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, TABLES, LOCK_ID); - - task.run(); - - verify(mockMetaStoreClient).heartbeat(0, LOCK_ID); - } - - @Test - public void testRunHeartbeatFailsNoSuchLockException() throws Exception { - NoSuchLockException exception = new NoSuchLockException(); - doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - - task.run(); - - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception); - } - - @Test - public void testRunHeartbeatFailsNoSuchTxnException() throws Exception { - NoSuchTxnException exception = new NoSuchTxnException(); - doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - - task.run(); - - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception); - } - - @Test - public void testRunHeartbeatFailsTxnAbortedException() throws Exception { - TxnAbortedException exception = new TxnAbortedException(); - doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - - task.run(); - - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception); - } - - @Test - public void testRunHeartbeatFailsTException() throws Exception { - TException exception = new TException(); - doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - - task.run(); - } - - private static List
createTable() { - Table table = new Table(); - table.setDbName("DB"); - table.setTableName("TABLE"); - return Arrays.asList(table); - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java deleted file mode 100644 index 156a37ebc7..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java +++ /dev/null @@ -1,339 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.client.lock; - -import static org.apache.hadoop.hive.metastore.api.LockState.ABORT; -import static org.apache.hadoop.hive.metastore.api.LockState.ACQUIRED; -import static org.apache.hadoop.hive.metastore.api.LockState.NOT_ACQUIRED; -import static org.apache.hadoop.hive.metastore.api.LockState.WAITING; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyCollection; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.lenient; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.verifyNoInteractions; -import static org.mockito.Mockito.when; - -import java.net.InetAddress; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.Timer; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.DataOperationType; -import org.apache.hadoop.hive.metastore.api.LockComponent; -import org.apache.hadoop.hive.metastore.api.LockLevel; -import org.apache.hadoop.hive.metastore.api.LockRequest; -import org.apache.hadoop.hive.metastore.api.LockResponse; -import org.apache.hadoop.hive.metastore.api.LockType; -import org.apache.hadoop.hive.metastore.api.NoSuchLockException; -import org.apache.hadoop.hive.metastore.api.NoSuchTxnException; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.TxnAbortedException; -import org.apache.thrift.TException; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -import com.google.common.collect.ImmutableSet; - -@RunWith(MockitoJUnitRunner.class) -public class TestLock { - - private static final Table SOURCE_TABLE_1 = createTable("DB", "SOURCE_1"); - private static final Table SOURCE_TABLE_2 = createTable("DB", "SOURCE_2"); - private static final Table SINK_TABLE = createTable("DB", "SINK"); - private static final Set
SOURCES = ImmutableSet.of(SOURCE_TABLE_1, SOURCE_TABLE_2); - private static final Set
SINKS = ImmutableSet.of(SINK_TABLE); - private static final Set
TABLES = ImmutableSet.of(SOURCE_TABLE_1, SOURCE_TABLE_2, SINK_TABLE); - private static final long LOCK_ID = 42; - private static final long TRANSACTION_ID = 109; - private static final String USER = "ewest"; - - @Mock - private IMetaStoreClient mockMetaStoreClient; - @Mock - private LockFailureListener mockListener; - @Mock - private LockResponse mockLockResponse; - @Mock - private HeartbeatFactory mockHeartbeatFactory; - @Mock - private Timer mockHeartbeat; - @Captor - private ArgumentCaptor requestCaptor; - - private Lock readLock; - private Lock writeLock; - private HiveConf configuration = new HiveConf(); - - @Before - public void injectMocks() throws Exception { - when(mockMetaStoreClient.lock(any(LockRequest.class))).thenReturn(mockLockResponse); - when(mockLockResponse.getLockid()).thenReturn(LOCK_ID); - when(mockLockResponse.getState()).thenReturn(ACQUIRED); - // Transaction IDs can also be null - when(mockHeartbeatFactory.newInstance( - any(IMetaStoreClient.class), any(LockFailureListener.class), any(), anyCollection(), any(Long.class), anyInt()) - ).thenReturn(mockHeartbeat); - - readLock = new Lock(mockMetaStoreClient, mockHeartbeatFactory, configuration, mockListener, USER, SOURCES, - Collections.
emptySet(), 3, 0); - writeLock = new Lock(mockMetaStoreClient, mockHeartbeatFactory, configuration, mockListener, USER, SOURCES, SINKS, - 3, 0); - } - - @Test - public void testAcquireReadLockWithNoIssues() throws Exception { - readLock.acquire(); - assertEquals(Long.valueOf(LOCK_ID), readLock.getLockId()); - assertNull(readLock.getTransactionId()); - } - - @Test(expected = IllegalArgumentException.class) - public void testAcquireWriteLockWithoutTxn() throws Exception { - writeLock.acquire(); - } - - @Test(expected = IllegalArgumentException.class) - public void testAcquireWriteLockWithInvalidTxn() throws Exception { - writeLock.acquire(0); - } - - @Test - public void testAcquireTxnLockWithNoIssues() throws Exception { - writeLock.acquire(TRANSACTION_ID); - assertEquals(Long.valueOf(LOCK_ID), writeLock.getLockId()); - assertEquals(Long.valueOf(TRANSACTION_ID), writeLock.getTransactionId()); - } - - @Test - public void testAcquireReadLockCheckHeartbeatCreated() throws Exception { - configuration.set("hive.txn.timeout", "100s"); - readLock.acquire(); - - verify(mockHeartbeatFactory).newInstance(eq(mockMetaStoreClient), eq(mockListener), any(), eq(SOURCES), - eq(LOCK_ID), eq(75)); - } - - @Test - public void testAcquireTxnLockCheckHeartbeatCreated() throws Exception { - configuration.set("hive.txn.timeout", "100s"); - writeLock.acquire(TRANSACTION_ID); - - verify(mockHeartbeatFactory).newInstance(eq(mockMetaStoreClient), eq(mockListener), eq(TRANSACTION_ID), - eq(TABLES), eq(LOCK_ID), eq(75)); - } - - @Test - public void testAcquireLockCheckUser() throws Exception { - readLock.acquire(); - verify(mockMetaStoreClient).lock(requestCaptor.capture()); - LockRequest actualRequest = requestCaptor.getValue(); - assertEquals(USER, actualRequest.getUser()); - } - - @Test - public void testAcquireReadLockCheckLocks() throws Exception { - readLock.acquire(); - verify(mockMetaStoreClient).lock(requestCaptor.capture()); - - LockRequest request = requestCaptor.getValue(); - assertEquals(0, request.getTxnid()); - assertEquals(USER, request.getUser()); - assertEquals(InetAddress.getLocalHost().getHostName(), request.getHostname()); - - List components = request.getComponent(); - - assertEquals(2, components.size()); - - LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB"); - expected1.setTablename("SOURCE_1"); - expected1.setOperationType(DataOperationType.INSERT); - expected1.setIsTransactional(true); - assertTrue(components.contains(expected1)); - - LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB"); - expected2.setTablename("SOURCE_2"); - expected2.setOperationType(DataOperationType.INSERT); - expected2.setIsTransactional(true); - assertTrue(components.contains(expected2)); - } - - @Test - public void testAcquireTxnLockCheckLocks() throws Exception { - writeLock.acquire(TRANSACTION_ID); - verify(mockMetaStoreClient).lock(requestCaptor.capture()); - - LockRequest request = requestCaptor.getValue(); - assertEquals(TRANSACTION_ID, request.getTxnid()); - assertEquals(USER, request.getUser()); - assertEquals(InetAddress.getLocalHost().getHostName(), request.getHostname()); - - List components = request.getComponent(); - - assertEquals(3, components.size()); - - LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB"); - expected1.setTablename("SOURCE_1"); - expected1.setOperationType(DataOperationType.INSERT); - expected1.setIsTransactional(true); - assertTrue(components.contains(expected1)); - - LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB"); - expected2.setTablename("SOURCE_2"); - expected2.setOperationType(DataOperationType.INSERT); - expected2.setIsTransactional(true); - assertTrue(components.contains(expected2)); - - LockComponent expected3 = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "DB"); - expected3.setTablename("SINK"); - expected3.setOperationType(DataOperationType.UPDATE); - expected3.setIsTransactional(true); - assertTrue(components.contains(expected3)); - } - - @Test(expected = LockException.class) - public void testAcquireLockNotAcquired() throws Exception { - when(mockLockResponse.getState()).thenReturn(NOT_ACQUIRED); - readLock.acquire(); - } - - @Test(expected = LockException.class) - public void testAcquireLockAborted() throws Exception { - when(mockLockResponse.getState()).thenReturn(ABORT); - readLock.acquire(); - } - - @Test(expected = LockException.class) - public void testAcquireLockWithWaitRetriesExceeded() throws Exception { - when(mockLockResponse.getState()).thenReturn(WAITING, WAITING, WAITING); - readLock.acquire(); - } - - @Test - public void testAcquireLockWithWaitRetries() throws Exception { - when(mockLockResponse.getState()).thenReturn(WAITING, WAITING, ACQUIRED); - readLock.acquire(); - assertEquals(Long.valueOf(LOCK_ID), readLock.getLockId()); - } - - @Test - public void testReleaseLock() throws Exception { - readLock.acquire(); - readLock.release(); - verify(mockMetaStoreClient).unlock(LOCK_ID); - } - - @Test - public void testReleaseLockNoLock() throws Exception { - readLock.release(); - verifyNoMoreInteractions(mockMetaStoreClient); - } - - @Test - public void testReleaseLockCancelsHeartbeat() throws Exception { - readLock.acquire(); - readLock.release(); - verify(mockHeartbeat).cancel(); - } - - @Test - public void testReadHeartbeat() throws Exception { - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, SOURCES, LOCK_ID); - task.run(); - verify(mockMetaStoreClient).heartbeat(0, LOCK_ID); - } - - @Test - public void testTxnHeartbeat() throws Exception { - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, SOURCES, - LOCK_ID); - task.run(); - verify(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - } - - @Test - public void testReadHeartbeatFailsNoSuchLockException() throws Exception { - Throwable t = new NoSuchLockException(); - doThrow(t).when(mockMetaStoreClient).heartbeat(0, LOCK_ID); - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, SOURCES, LOCK_ID); - task.run(); - verify(mockListener).lockFailed(LOCK_ID, null, Lock.asStrings(SOURCES), t); - } - - @Test - public void testTxnHeartbeatFailsNoSuchLockException() throws Exception { - Throwable t = new NoSuchLockException(); - doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, SOURCES, - LOCK_ID); - task.run(); - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(SOURCES), t); - } - - @Test - public void testHeartbeatFailsNoSuchTxnException() throws Exception { - Throwable t = new NoSuchTxnException(); - doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, SOURCES, - LOCK_ID); - task.run(); - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(SOURCES), t); - } - - @Test - public void testHeartbeatFailsTxnAbortedException() throws Exception { - Throwable t = new TxnAbortedException(); - doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID); - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, SOURCES, - LOCK_ID); - task.run(); - verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(SOURCES), t); - } - - @Test - public void testHeartbeatContinuesTException() throws Exception { - Throwable t = new TException(); - lenient().doThrow(t).when(mockMetaStoreClient).heartbeat(0, LOCK_ID); - HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, SOURCES, - LOCK_ID); - task.run(); - verifyNoInteractions(mockListener); - } - - private static Table createTable(String databaseName, String tableName) { - Table table = new Table(); - table.setDbName(databaseName); - table.setTableName(tableName); - return table; - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java deleted file mode 100644 index e890c52d6d..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; - -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.BucketCodec; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hive.hcatalog.streaming.mutate.MutableRecord; -import org.junit.Test; - -public class TestBucketIdResolverImpl { - - private static final int TOTAL_BUCKETS = 12; - private static final int RECORD_ID_COLUMN = 2; - // id - TODO: use a non-zero index to check for offset errors. - private static final int[] BUCKET_COLUMN_INDEXES = new int[] { 0 }; - - private BucketIdResolver capturingBucketIdResolver = new BucketIdResolverImpl( - ObjectInspectorFactory.getReflectionObjectInspector(MutableRecord.class, - ObjectInspectorFactory.ObjectInspectorOptions.JAVA), RECORD_ID_COLUMN, TOTAL_BUCKETS, BUCKET_COLUMN_INDEXES); - - @Test - public void testAttachBucketIdToRecord() { - MutableRecord record = new MutableRecord(1, "hello"); - capturingBucketIdResolver.attachBucketIdToRecord(record); - assertThat(record.rowId, is(new RecordIdentifier(-1L, - BucketCodec.V1.encode(new AcidOutputFormat.Options(null).bucket(1)), - -1L))); - assertThat(record.id, is(1)); - assertThat(record.msg.toString(), is("hello")); - } - - @Test(expected = IllegalArgumentException.class) - public void testNoBucketColumns() { - new BucketIdResolverImpl(ObjectInspectorFactory.getReflectionObjectInspector(MutableRecord.class, - ObjectInspectorFactory.ObjectInspectorOptions.JAVA), RECORD_ID_COLUMN, TOTAL_BUCKETS, new int[0]); - - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java deleted file mode 100644 index 1d171c401e..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.Arrays; -import java.util.Collections; - -import org.junit.Test; - -public class TestGroupingValidator { - - private GroupingValidator validator = new GroupingValidator(); - - @Test - public void uniqueGroups() { - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3)); - assertTrue(validator.isInSequence(Arrays.asList("b", "B"), 2)); - } - - @Test - public void sameGroup() { - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - } - - @Test - public void revisitedGroup() { - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3)); - assertFalse(validator.isInSequence(Arrays.asList("a", "A"), 1)); - } - - @Test - public void samePartitionDifferentBucket() { - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3)); - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 2)); - } - - @Test - public void sameBucketDifferentPartition() { - assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1)); - assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3)); - assertTrue(validator.isInSequence(Arrays.asList("b", "B"), 1)); - } - - @Test - public void uniqueGroupsNoPartition() { - assertTrue(validator.isInSequence(Collections. emptyList(), 1)); - assertTrue(validator.isInSequence(Collections. emptyList(), 3)); - assertTrue(validator.isInSequence(Collections. emptyList(), 2)); - } - - @Test - public void sameGroupNoPartition() { - assertTrue(validator.isInSequence(Collections. emptyList(), 1)); - assertTrue(validator.isInSequence(Collections. emptyList(), 1)); - assertTrue(validator.isInSequence(Collections. emptyList(), 1)); - } - - @Test - public void revisitedGroupNoPartition() { - assertTrue(validator.isInSequence(Collections. emptyList(), 1)); - assertTrue(validator.isInSequence(Collections. emptyList(), 3)); - assertFalse(validator.isInSequence(Collections. emptyList(), 1)); - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java deleted file mode 100644 index 4a7d358698..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.metastore.api.Table; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestMetaStorePartitionHelper { - - private static final Path TABLE_PATH = new Path("table"); - private static final String TABLE_LOCATION = TABLE_PATH.toString(); - - private static final FieldSchema PARTITION_KEY_A = new FieldSchema("A", "string", null); - private static final FieldSchema PARTITION_KEY_B = new FieldSchema("B", "string", null); - private static final List PARTITION_KEYS = Arrays.asList(PARTITION_KEY_A, PARTITION_KEY_B); - private static final Path PARTITION_PATH = new Path(TABLE_PATH, "a=1/b=2"); - private static final String PARTITION_LOCATION = PARTITION_PATH.toString(); - - private static final String DATABASE_NAME = "db"; - private static final String TABLE_NAME = "one"; - - private static final List UNPARTITIONED_VALUES = Collections.emptyList(); - private static final List PARTITIONED_VALUES = Arrays.asList("1", "2"); - - @Mock - private IMetaStoreClient mockClient; - @Mock - private Table mockTable; - private StorageDescriptor tableStorageDescriptor = new StorageDescriptor(); - - @Mock - private Partition mockPartition; - @Mock - private StorageDescriptor mockPartitionStorageDescriptor; - @Captor - private ArgumentCaptor partitionCaptor; - - private PartitionHelper helper; - - @Before - public void injectMocks() throws Exception { - when(mockClient.getTable(DATABASE_NAME, TABLE_NAME)).thenReturn(mockTable); - when(mockTable.getDbName()).thenReturn(DATABASE_NAME); - when(mockTable.getTableName()).thenReturn(TABLE_NAME); - when(mockTable.getPartitionKeys()).thenReturn(PARTITION_KEYS); - when(mockTable.getSd()).thenReturn(tableStorageDescriptor); - tableStorageDescriptor.setLocation(TABLE_LOCATION); - - when(mockClient.getPartition(DATABASE_NAME, TABLE_NAME, PARTITIONED_VALUES)).thenReturn(mockPartition); - when(mockPartition.getSd()).thenReturn(mockPartitionStorageDescriptor); - when(mockPartitionStorageDescriptor.getLocation()).thenReturn(PARTITION_LOCATION); - - helper = new MetaStorePartitionHelper(mockClient, DATABASE_NAME, TABLE_NAME, TABLE_PATH); - } - - @Test - public void getPathForUnpartitionedTable() throws Exception { - Path path = helper.getPathForPartition(UNPARTITIONED_VALUES); - assertThat(path, is(TABLE_PATH)); - verifyNoInteractions(mockClient); - } - - @Test - public void getPathForPartitionedTable() throws Exception { - Path path = helper.getPathForPartition(PARTITIONED_VALUES); - assertThat(path, is(PARTITION_PATH)); - } - - @Test - public void createOnUnpartitionTableDoesNothing() throws Exception { - helper.createPartitionIfNotExists(UNPARTITIONED_VALUES); - verifyNoInteractions(mockClient); - } - - @Test - public void createOnPartitionTable() throws Exception { - helper.createPartitionIfNotExists(PARTITIONED_VALUES); - - verify(mockClient).add_partition(partitionCaptor.capture()); - Partition actual = partitionCaptor.getValue(); - assertThat(actual.getSd().getLocation(), is(PARTITION_LOCATION)); - assertThat(actual.getValues(), is(PARTITIONED_VALUES)); - } - - @Test - public void closeSucceeds() throws IOException { - helper.close(); - verify(mockClient).close(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java deleted file mode 100644 index 297e67c40d..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java +++ /dev/null @@ -1,262 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.lenient; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; -import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestMutatorCoordinator { - - private static final List UNPARTITIONED = Collections. emptyList(); - private static final List PARTITION_B = Arrays.asList("B"); - private static final List PARTITION_A = Arrays.asList("A"); - private static final long WRITE_ID = 2L; - private static final int BUCKET_ID = 0; - private static final Path PATH_A = new Path("X"); - private static final Path PATH_B = new Path("B"); - private static final Object RECORD = "RECORD"; - private static final RecordIdentifier ROW__ID_B0_R0 = new RecordIdentifier(10L, BUCKET_ID, 0L); - private static final RecordIdentifier ROW__ID_B0_R1 = new RecordIdentifier(10L, BUCKET_ID, 1L); - private static final RecordIdentifier ROW__ID_B1_R0 = new RecordIdentifier(10L, BUCKET_ID + 1, 0L); - private static final RecordIdentifier ROW__ID_INSERT = new RecordIdentifier(-1L, BUCKET_ID, -1L); - - @Mock - private MutatorFactory mockMutatorFactory; - @Mock - private PartitionHelper mockPartitionHelper; - @Mock - private GroupingValidator mockGroupingValidator; - @Mock - private SequenceValidator mockSequenceValidator; - @Mock - private AcidTable mockAcidTable; - @Mock - private RecordInspector mockRecordInspector; - @Mock - private BucketIdResolver mockBucketIdResolver; - @Mock - private Mutator mockMutator; - - private MutatorCoordinator coordinator; - - private HiveConf configuration = new HiveConf(); - - @Before - public void createCoordinator() throws Exception { - when(mockAcidTable.getOutputFormatName()).thenReturn(OrcOutputFormat.class.getName()); - when(mockAcidTable.getTotalBuckets()).thenReturn(1); - when(mockAcidTable.getWriteId()).thenReturn(WRITE_ID); - when(mockAcidTable.createPartitions()).thenReturn(true); - when(mockMutatorFactory.newRecordInspector()).thenReturn(mockRecordInspector); - when(mockMutatorFactory.newBucketIdResolver(anyInt())).thenReturn(mockBucketIdResolver); - when(mockMutatorFactory.newMutator(any(OrcOutputFormat.class), anyLong(), any(Path.class), anyInt())).thenReturn( - mockMutator); - when(mockPartitionHelper.getPathForPartition(any(List.class))).thenReturn(PATH_A); - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_INSERT); - lenient().when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(true); - when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(true); - - coordinator = new MutatorCoordinator(configuration, mockMutatorFactory, mockPartitionHelper, mockGroupingValidator, - mockSequenceValidator, mockAcidTable, false); - } - - @Test - public void insert() throws Exception { - coordinator.insert(UNPARTITIONED, RECORD); - - verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutator).insert(RECORD); - } - - @Test - public void multipleInserts() throws Exception { - coordinator.insert(UNPARTITIONED, RECORD); - coordinator.insert(UNPARTITIONED, RECORD); - coordinator.insert(UNPARTITIONED, RECORD); - - verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutator, times(3)).insert(RECORD); - } - - @Test - public void insertPartitionChanges() throws Exception { - when(mockPartitionHelper.getPathForPartition(PARTITION_A)).thenReturn(PATH_A); - when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B); - - coordinator.insert(PARTITION_A, RECORD); - coordinator.insert(PARTITION_B, RECORD); - - verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_A); - verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_B); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_B), eq(BUCKET_ID)); - verify(mockMutator, times(2)).insert(RECORD); - } - - @Test - public void bucketChanges() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_B1_R0); - - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 1); - - coordinator.update(UNPARTITIONED, RECORD); - coordinator.delete(UNPARTITIONED, RECORD); - - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutatorFactory) - .newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID + 1)); - verify(mockMutator).update(RECORD); - verify(mockMutator).delete(RECORD); - } - - @Test - public void partitionThenBucketChanges() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_B0_R1, ROW__ID_B1_R0, - ROW__ID_INSERT); - - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 0, 1, 0); - - when(mockPartitionHelper.getPathForPartition(PARTITION_A)).thenReturn(PATH_A); - when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B); - - coordinator.update(PARTITION_A, RECORD); /* PaB0 */ - coordinator.insert(PARTITION_B, RECORD); /* PbB0 */ - coordinator.delete(PARTITION_B, RECORD); /* PbB0 */ - coordinator.update(PARTITION_B, RECORD); /* PbB1 */ - - verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_B); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutatorFactory, times(2)).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_B), - eq(BUCKET_ID)); - verify(mockMutatorFactory) - .newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_B), eq(BUCKET_ID + 1)); - verify(mockMutator, times(2)).update(RECORD); - verify(mockMutator).delete(RECORD); - verify(mockMutator).insert(RECORD); - verify(mockSequenceValidator, times(4)).reset(); - } - - @Test - public void partitionThenBucketChangesNoCreateAsPartitionEstablished() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_INSERT); - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 0); - when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B); - - coordinator.delete(PARTITION_B, RECORD); /* PbB0 */ - coordinator.insert(PARTITION_B, RECORD); /* PbB0 */ - - verify(mockPartitionHelper, never()).createPartitionIfNotExists(anyList()); - } - - @Test(expected = RecordSequenceException.class) - public void outOfSequence() throws Exception { - when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(false); - - coordinator.update(UNPARTITIONED, RECORD); - coordinator.delete(UNPARTITIONED, RECORD); - - verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutator).update(RECORD); - verify(mockMutator).delete(RECORD); - } - - @Test(expected = GroupRevisitedException.class) - public void revisitGroup() throws Exception { - when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(false); - - coordinator.update(UNPARTITIONED, RECORD); - coordinator.delete(UNPARTITIONED, RECORD); - - verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED); - verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(WRITE_ID), eq(PATH_A), eq(BUCKET_ID)); - verify(mockMutator).update(RECORD); - verify(mockMutator).delete(RECORD); - } - - @Test(expected = BucketIdException.class) - public void insertWithBadBucket() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0); - - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1); - - coordinator.insert(UNPARTITIONED, RECORD); - } - - @Test(expected = BucketIdException.class) - public void updateWithBadBucket() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0); - - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1); - - coordinator.update(UNPARTITIONED, RECORD); - } - - @Test - public void deleteWithBadBucket() throws Exception { - when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0); - - when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1); - - coordinator.delete(UNPARTITIONED, RECORD); - } - - @Test - public void closeNoRecords() throws Exception { - coordinator.close(); - - // No mutator created - verifyNoMoreInteractions(mockMutator); - } - - @Test - public void closeUsedCoordinator() throws Exception { - coordinator.insert(UNPARTITIONED, RECORD); - coordinator.close(); - - verify(mockMutator).close(); - verify(mockPartitionHelper).close(); - } -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java deleted file mode 100644 index 5219532d3e..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat; -import org.apache.hadoop.hive.ql.io.AcidOutputFormat.Options; -import org.apache.hadoop.hive.ql.io.RecordUpdater; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TestMutatorImpl { - - private static final Object RECORD = new Object(); - private static final int RECORD_ID_COLUMN = 2; - private static final int BUCKET_ID = 0; - private static final Path PATH = new Path("X"); - private static final long WRITE_ID = 1L; - - @Mock - private AcidOutputFormat mockOutputFormat; - @Mock - private ObjectInspector mockObjectInspector; - @Mock - private RecordUpdater mockRecordUpdater; - @Captor - private ArgumentCaptor captureOptions; - - private final HiveConf configuration = new HiveConf(); - - private Mutator mutator; - - @Before - public void injectMocks() throws IOException { - when(mockOutputFormat.getRecordUpdater(eq(PATH), any(Options.class))).thenReturn(mockRecordUpdater); - mutator = new MutatorImpl(configuration, RECORD_ID_COLUMN, mockObjectInspector, mockOutputFormat, WRITE_ID, - PATH, BUCKET_ID); - } - - @Test - public void testCreatesRecordReader() throws IOException { - verify(mockOutputFormat).getRecordUpdater(eq(PATH), captureOptions.capture()); - Options options = captureOptions.getValue(); - assertThat(options.getBucketId(), is(BUCKET_ID)); - assertThat(options.getConfiguration(), is((Configuration) configuration)); - assertThat(options.getInspector(), is(mockObjectInspector)); - assertThat(options.getRecordIdColumn(), is(RECORD_ID_COLUMN)); - assertThat(options.getMinimumWriteId(), is(WRITE_ID)); - assertThat(options.getMaximumWriteId(), is(WRITE_ID)); - } - - @Test - public void testInsertDelegates() throws IOException { - mutator.insert(RECORD); - verify(mockRecordUpdater).insert(WRITE_ID, RECORD); - } - - @Test - public void testUpdateDelegates() throws IOException { - mutator.update(RECORD); - verify(mockRecordUpdater).update(WRITE_ID, RECORD); - } - - @Test - public void testDeleteDelegates() throws IOException { - mutator.delete(RECORD); - verify(mockRecordUpdater).delete(WRITE_ID, RECORD); - } - - @Test - public void testCloseDelegates() throws IOException { - mutator.close(); - verify(mockRecordUpdater).close(false); - } - - @Test - public void testFlushDoesNothing() throws IOException { - mutator.flush(); - verify(mockRecordUpdater, never()).flush(); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java deleted file mode 100644 index 55da312a5a..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; - -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; -import org.apache.hive.hcatalog.streaming.mutate.MutableRecord; -import org.junit.Test; - -public class TestRecordInspectorImpl { - - private static final int ROW_ID_COLUMN = 2; - - private RecordInspectorImpl inspector = new RecordInspectorImpl(ObjectInspectorFactory.getReflectionObjectInspector( - MutableRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA), ROW_ID_COLUMN); - - @Test - public void testExtractRecordIdentifier() { - RecordIdentifier recordIdentifier = new RecordIdentifier(10L, 4, 20L); - MutableRecord record = new MutableRecord(1, "hello", recordIdentifier); - assertThat(inspector.extractRecordIdentifier(record), is(recordIdentifier)); - } - - @Test(expected = IllegalArgumentException.class) - public void testNotAStructObjectInspector() { - new RecordInspectorImpl(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, 2); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java deleted file mode 100644 index 2b3f79fcc0..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; - -import org.apache.hadoop.hive.ql.io.RecordIdentifier; -import org.junit.Test; - -public class TestSequenceValidator { - - private static final int BUCKET_ID = 1; - - private SequenceValidator validator = new SequenceValidator(); - - @Test - public void testSingleInSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - } - - @Test - public void testRowIdInSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true)); - } - - @Test - public void testTxIdInSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(4L, BUCKET_ID, 0)), is(true)); - } - - @Test - public void testMixedInSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 1)), is(true)); - } - - @Test - public void testNegativeTxId() { - assertThat(validator.isInSequence(new RecordIdentifier(-1L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - } - - @Test - public void testNegativeRowId() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, -1)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - } - - @Test - public void testRowIdOutOfSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(false)); - } - - @Test - public void testReset() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true)); - // New partition for example - validator.reset(); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true)); - } - - @Test - public void testTxIdOutOfSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(4L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(false)); - } - - @Test - public void testMixedOutOfSequence() { - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 4)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(false)); - assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 5)), is(true)); - assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 6)), is(false)); - } - - @Test(expected = NullPointerException.class) - public void testNullRecordIdentifier() { - validator.isInSequence(null); - } - -} diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java deleted file mode 100644 index 1011d34a61..0000000000 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hive.hcatalog.streaming.mutate.worker; - -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.junit.Test; - -public class TestWarehousePartitionHelper { - - private static final HiveConf CONFIGURATION = new HiveConf(); - private static final Path TABLE_PATH = new Path("table"); - - private static final List UNPARTITIONED_COLUMNS = Collections.emptyList(); - private static final List UNPARTITIONED_VALUES = Collections.emptyList(); - - private static final List PARTITIONED_COLUMNS = Arrays.asList("A", "B"); - private static final List PARTITIONED_VALUES = Arrays.asList("1", "2"); - - private final PartitionHelper unpartitionedHelper; - private final PartitionHelper partitionedHelper; - - public TestWarehousePartitionHelper() throws Exception { - unpartitionedHelper = new WarehousePartitionHelper(CONFIGURATION, TABLE_PATH, UNPARTITIONED_COLUMNS); - partitionedHelper = new WarehousePartitionHelper(CONFIGURATION, TABLE_PATH, PARTITIONED_COLUMNS); - } - - @Test(expected = UnsupportedOperationException.class) - public void createNotSupported() throws Exception { - unpartitionedHelper.createPartitionIfNotExists(UNPARTITIONED_VALUES); - } - - @Test - public void getPathForUnpartitionedTable() throws Exception { - Path path = unpartitionedHelper.getPathForPartition(UNPARTITIONED_VALUES); - assertThat(path, is(TABLE_PATH)); - } - - @Test - public void getPathForPartitionedTable() throws Exception { - Path path = partitionedHelper.getPathForPartition(PARTITIONED_VALUES); - assertThat(path, is(new Path(TABLE_PATH, "A=1/B=2"))); - } - - @Test - public void closeSucceeds() throws IOException { - partitionedHelper.close(); - unpartitionedHelper.close(); - } - -} diff --git a/hcatalog/streaming/src/test/sit b/hcatalog/streaming/src/test/sit deleted file mode 100644 index 38cc3525e6..0000000000 --- a/hcatalog/streaming/src/test/sit +++ /dev/null @@ -1,39 +0,0 @@ -#!/bin/sh - -if [ "${HADOOP_HOME}x" == "x" ] - then - echo "Please set HADOOP_HOME"; - exit 1 -fi - -if [ "${HIVE_HOME}x" == "x" ] - then - echo "Please set HIVE_HOME"; - exit 1 -fi - -if [ "${JAVA_HOME}x" == "x" ] - then - echo "Please set JAVA_HOME"; - exit 1 -fi - -for jar in ${HADOOP_HOME}/client/*.jar - do - CLASSPATH=${CLASSPATH}:$jar -done - -for jar in ${HIVE_HOME}/lib/*.jar - do - CLASSPATH=${CLASSPATH}:$jar -done - -for jar in ${HIVE_HOME}/hcatalog/share/hcatalog/*.jar - do - CLASSPATH=${CLASSPATH}:$jar -done - -CLASSPATH=${CLASSPATH}:${HADOOP_HOME}/etc/hadoop -CLASSPATH=${CLASSPATH}:${HIVE_HOME}/conf - -$JAVA_HOME/bin/java -cp ${CLASSPATH} org.apache.hive.hcatalog.streaming.StreamingIntegrationTester $@ diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index 026c465655..43e3bfb13f 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -76,10 +76,6 @@ org.apache.hive.hcatalog hive-hcatalog-core - - org.apache.hive.hcatalog - hive-hcatalog-streaming - org.apache.hive hive-streaming diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java index e70d8783bc..024a71b2ac 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java @@ -37,8 +37,6 @@ import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcStruct; import org.apache.hadoop.hive.ql.processors.CommandProcessorException; -import org.apache.hive.hcatalog.streaming.DelimitedInputWriter; -import org.apache.hive.hcatalog.streaming.TransactionBatch; import org.apache.hive.streaming.HiveStreamingConnection; import org.apache.hive.streaming.StreamingConnection; import org.apache.hive.streaming.StreamingException; @@ -260,25 +258,6 @@ static StreamingConnection writeBatch(HiveConf conf, String dbName, String tblNa return connection; } - static void writeBatch(org.apache.hive.hcatalog.streaming.StreamingConnection connection, - DelimitedInputWriter writer, - boolean closeEarly) throws InterruptedException, org.apache.hive.hcatalog.streaming.StreamingException { - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.write("50,Kiev".getBytes()); - txnBatch.write("51,St. Petersburg".getBytes()); - txnBatch.write("44,Boston".getBytes()); - txnBatch.commit(); - if (!closeEarly) { - txnBatch.beginNextTransaction(); - txnBatch.write("52,Tel Aviv".getBytes()); - txnBatch.write("53,Atlantis".getBytes()); - txnBatch.write("53,Boston".getBytes()); - txnBatch.commit(); - txnBatch.close(); - } - } - static void checkExpectedTxnsPresent(Path base, Path[] deltas, String columnNamesProperty, String columnTypesProperty, int bucket, long min, long max, List invaliWriteIDs, int numBuckets) throws IOException { ValidWriteIdList writeIdList = new ValidWriteIdList() { diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index 32fe535b2b..8fdbabe437 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -26,7 +26,15 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -65,9 +73,6 @@ import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hive.common.util.Retry; import org.apache.hive.hcatalog.common.HCatUtil; -import org.apache.hive.hcatalog.streaming.DelimitedInputWriter; -import org.apache.hive.hcatalog.streaming.HiveEndPoint; -import org.apache.hive.hcatalog.streaming.TransactionBatch; import org.apache.hive.streaming.HiveStreamingConnection; import org.apache.hive.streaming.StreamingConnection; import org.apache.hive.streaming.StreamingException; @@ -98,11 +103,6 @@ private final String BASIC_FILE_NAME = TEST_DATA_DIR + "/basic.input.data"; private final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; - @Parameterized.Parameters - public static Collection data() { - return Arrays.asList(new Object[][]{{true}, {false}}); - } - @Rule public TemporaryFolder stagingFolder = new TemporaryFolder(); @@ -346,13 +346,6 @@ public void schemaEvolutionAddColDynamicPartitioningUpdate() throws Exception { */ @Test public void testStatsAfterCompactionPartTbl() throws Exception { - testStatsAfterCompactionPartTbl(false); - } - @Test - public void testStatsAfterCompactionPartTblNew() throws Exception { - testStatsAfterCompactionPartTbl(true); - } - private void testStatsAfterCompactionPartTbl(boolean newStreamingAPI) throws Exception { //as of (8/27/2014) Hive 0.14, ACID/Orc requires HiveInputFormat String tblName = "compaction_test"; String tblNameStg = tblName + "_stg"; @@ -420,58 +413,29 @@ private void testStatsAfterCompactionPartTbl(boolean newStreamingAPI) throws Exc LongColumnStatsData colAStatsPart2 = colStats.get(0).getStatsData().getLongStats(); StringColumnStatsData colBStatsPart2 = colStats.get(1).getStatsData().getStringStats(); - if (newStreamingAPI) { - StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder() - .withFieldDelimiter(',') - .build(); - HiveStreamingConnection connection = HiveStreamingConnection.newBuilder() - .withDatabase(ci.dbname) - .withTable(ci.tableName) - .withStaticPartitionValues(Arrays.asList("0")) - .withAgentInfo("UT_" + Thread.currentThread().getName()) - .withHiveConf(conf) - .withRecordWriter(writer) - .connect(); - connection.beginTransaction(); - connection.write("50,Kiev".getBytes()); - connection.write("51,St. Petersburg".getBytes()); - connection.write("44,Boston".getBytes()); - connection.commitTransaction(); - - connection.beginTransaction(); - connection.write("52,Tel Aviv".getBytes()); - connection.write("53,Atlantis".getBytes()); - connection.write("53,Boston".getBytes()); - connection.commitTransaction(); - connection.close(); - } else { - HiveEndPoint endPt = new HiveEndPoint(null, ci.dbname, ci.tableName, Arrays.asList("0")); - DelimitedInputWriter writer = new DelimitedInputWriter(new String[]{"a", "b"}, ",", endPt); - /*next call will eventually end up in HiveEndPoint.createPartitionIfNotExists() which - makes an operation on Driver - * and starts it's own CliSessionState and then closes it, which removes it from ThreadLoacal; - * thus the session - * created in this class is gone after this; I fixed it in HiveEndPoint*/ - org.apache.hive.hcatalog.streaming.StreamingConnection connection = endPt - .newConnection(true, "UT_" + Thread.currentThread().getName()); - - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - Assert.assertEquals(TransactionBatch.TxnState.OPEN, txnBatch.getCurrentTransactionState()); - txnBatch.write("50,Kiev".getBytes()); - txnBatch.write("51,St. Petersburg".getBytes()); - txnBatch.write("44,Boston".getBytes()); - txnBatch.commit(); - - txnBatch.beginNextTransaction(); - txnBatch.write("52,Tel Aviv".getBytes()); - txnBatch.write("53,Atlantis".getBytes()); - txnBatch.write("53,Boston".getBytes()); - txnBatch.commit(); - - txnBatch.close(); - connection.close(); - } + StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder() + .withFieldDelimiter(',') + .build(); + HiveStreamingConnection connection = HiveStreamingConnection.newBuilder() + .withDatabase(ci.dbname) + .withTable(ci.tableName) + .withStaticPartitionValues(Arrays.asList("0")) + .withAgentInfo("UT_" + Thread.currentThread().getName()) + .withHiveConf(conf) + .withRecordWriter(writer) + .connect(); + connection.beginTransaction(); + connection.write("50,Kiev".getBytes()); + connection.write("51,St. Petersburg".getBytes()); + connection.write("44,Boston".getBytes()); + connection.commitTransaction(); + + connection.beginTransaction(); + connection.write("52,Tel Aviv".getBytes()); + connection.write("53,Atlantis".getBytes()); + connection.write("53,Boston".getBytes()); + connection.commitTransaction(); + connection.close(); execSelectAndDumpData("select * from " + ci.getFullTableName(), driver, ci.getFullTableName()); //so now we have written some new data to bkt=0 and it shows up @@ -699,15 +663,12 @@ public void majorCompactWhileStreaming() throws Exception { } } - @Test - public void minorCompactAfterAbort() throws Exception { - minorCompactAfterAbort(false); - } @Test public void minorCompactAfterAbortNew() throws Exception { - minorCompactAfterAbort(true); + minorCompactAfterAbort(); } - private void minorCompactAfterAbort(boolean newStreamingAPI) throws Exception { + + private void minorCompactAfterAbort() throws Exception { String dbName = "default"; String tblName = "cws"; String columnNamesProperty = "a,b"; @@ -717,7 +678,8 @@ private void minorCompactAfterAbort(boolean newStreamingAPI) throws Exception { " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver); - processStreamingAPI(dbName, tblName, newStreamingAPI); + processStreamingAPI(dbName, tblName); + // Now, compact TxnStore txnHandler = TxnUtils.getTxnStore(conf); txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR)); @@ -750,13 +712,6 @@ private void minorCompactAfterAbort(boolean newStreamingAPI) throws Exception { @Test public void majorCompactAfterAbort() throws Exception { - majorCompactAfterAbort(false); - } - @Test - public void majorCompactAfterAbortNew() throws Exception { - majorCompactAfterAbort(true); - } - private void majorCompactAfterAbort(boolean newStreamingAPI) throws Exception { String dbName = "default"; String tblName = "cws"; String columnNamesProperty = "a,b"; @@ -766,7 +721,7 @@ private void majorCompactAfterAbort(boolean newStreamingAPI) throws Exception { " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver); - processStreamingAPI(dbName, tblName, newStreamingAPI); + processStreamingAPI(dbName, tblName); runMajorCompaction(dbName, tblName); // Find the location of the table @@ -1136,13 +1091,6 @@ private void runMajorCompaction( @Test public void majorCompactWhileStreamingForSplitUpdate() throws Exception { - majorCompactWhileStreamingForSplitUpdate(false); - } - @Test - public void majorCompactWhileStreamingForSplitUpdateNew() throws Exception { - majorCompactWhileStreamingForSplitUpdate(true); - } - private void majorCompactWhileStreamingForSplitUpdate(boolean newStreamingAPI) throws Exception { String dbName = "default"; String tblName = "cws"; String columnNamesProperty = "a,b"; @@ -1153,29 +1101,14 @@ private void majorCompactWhileStreamingForSplitUpdate(boolean newStreamingAPI) t " STORED AS ORC TBLPROPERTIES ('transactional'='true', " + "'transactional_properties'='default') ", driver); // this turns on split-update U=D+I - StreamingConnection connection1 = null; - org.apache.hive.hcatalog.streaming.StreamingConnection connection2 = null; - if (newStreamingAPI) { - // Write a couple of batches - for (int i = 0; i < 2; i++) { - CompactorTestUtil.writeBatch(conf, dbName, tblName, false, false); - } + // Write a couple of batches + for (int i = 0; i < 2; i++) { + CompactorTestUtil.writeBatch(conf, dbName, tblName, false, false); + } - // Start a third batch, but don't close it. - connection1 = CompactorTestUtil.writeBatch(conf, dbName, tblName, false, true); - } else { - HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null); - DelimitedInputWriter writer = new DelimitedInputWriter(new String[]{"a", "b"}, ",", endPt); - connection2 = endPt - .newConnection(false, "UT_" + Thread.currentThread().getName()); - // Write a couple of batches - for (int i = 0; i < 2; i++) { - CompactorTestUtil.writeBatch(connection2, writer, false); - } + // Start a third batch, but don't close it. + StreamingConnection connection1 = CompactorTestUtil.writeBatch(conf, dbName, tblName, false, true); - // Start a third batch, but don't close it. - CompactorTestUtil.writeBatch(connection2, writer, true); - } runMajorCompaction(dbName, tblName); // Find the location of the table @@ -1195,9 +1128,6 @@ private void majorCompactWhileStreamingForSplitUpdate(boolean newStreamingAPI) t if (connection1 != null) { connection1.close(); } - if (connection2 != null) { - connection2.close(); - } } @Test @@ -1386,13 +1316,6 @@ public void testCompactionForFileInSratchDir() throws Exception { @Test public void minorCompactWhileStreamingWithSplitUpdate() throws Exception { - minorCompactWhileStreamingWithSplitUpdate(false); - } - @Test - public void minorCompactWhileStreamingWithSplitUpdateNew() throws Exception { - minorCompactWhileStreamingWithSplitUpdate(true); - } - private void minorCompactWhileStreamingWithSplitUpdate(boolean newStreamingAPI) throws Exception { String dbName = "default"; String tblName = "cws"; String columnNamesProperty = "a,b"; @@ -1403,30 +1326,13 @@ private void minorCompactWhileStreamingWithSplitUpdate(boolean newStreamingAPI) " STORED AS ORC TBLPROPERTIES ('transactional'='true'," + "'transactional_properties'='default')", driver); - StreamingConnection connection1 = null; - org.apache.hive.hcatalog.streaming.StreamingConnection connection2 = null; - if (newStreamingAPI) { - - // Write a couple of batches - for (int i = 0; i < 2; i++) { - CompactorTestUtil.writeBatch(conf, dbName, tblName, false, false); - } - - // Start a third batch, but don't close it. - connection1 = CompactorTestUtil.writeBatch(conf, dbName, tblName, false, true); - } else { - HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null); - DelimitedInputWriter writer = new DelimitedInputWriter(new String[]{"a", "b"}, ",", endPt); - connection2 = endPt - .newConnection(false, "UT_" + Thread.currentThread().getName()); - // Write a couple of batches - for (int i = 0; i < 2; i++) { - CompactorTestUtil.writeBatch(connection2, writer, false); - } - - // Start a third batch, but don't close it. - CompactorTestUtil.writeBatch(connection2, writer, true); + // Write a couple of batches + for (int i = 0; i < 2; i++) { + CompactorTestUtil.writeBatch(conf, dbName, tblName, false, false); } + + // Start a third batch, but don't close it. + StreamingConnection connection1 = CompactorTestUtil.writeBatch(conf, dbName, tblName, false, true); // Now, compact TxnStore txnHandler = TxnUtils.getTxnStore(conf); txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR)); @@ -1464,9 +1370,6 @@ private void minorCompactWhileStreamingWithSplitUpdate(boolean newStreamingAPI) if (connection1 != null) { connection1.close(); } - if (connection2 != null) { - connection2.close(); - } } @Test @@ -1931,35 +1834,13 @@ private void verifyCompactions(List compacts, Sorted } } - private void processStreamingAPI(String dbName, String tblName, boolean newStreamingAPI) + private void processStreamingAPI(String dbName, String tblName) throws StreamingException, ClassNotFoundException, - org.apache.hive.hcatalog.streaming.StreamingException, InterruptedException { - if (newStreamingAPI) { + InterruptedException { List options = Lists .newArrayList(new CompactorTestUtil.StreamingConnectionOption(false, false), new CompactorTestUtil.StreamingConnectionOption(false, false), new CompactorTestUtil.StreamingConnectionOption(true, false)); CompactorTestUtil.runStreamingAPI(conf, dbName, tblName, options); - } else { - HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null); - DelimitedInputWriter writer = new DelimitedInputWriter(new String[]{"a", "b"}, ",", endPt); - org.apache.hive.hcatalog.streaming.StreamingConnection connection = endPt - .newConnection(false, "UT_" + Thread.currentThread().getName()); - try { - // Write a couple of batches - for (int i = 0; i < 2; i++) { - CompactorTestUtil.writeBatch(connection, writer, false); - } - - // Start a third batch, abort everything, don't properly close it - TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer); - txnBatch.beginNextTransaction(); - txnBatch.abort(); - txnBatch.beginNextTransaction(); - txnBatch.abort(); - } finally { - connection.close(); - } - } } } diff --git a/packaging/pom.xml b/packaging/pom.xml index 97c8cf7168..4a632d4711 100644 --- a/packaging/pom.xml +++ b/packaging/pom.xml @@ -267,11 +267,6 @@ hive-streaming ${project.version} - - org.apache.hive.hcatalog - hive-hcatalog-streaming - ${project.version} - org.apache.hive.hcatalog hive-hcatalog-core diff --git a/packaging/src/main/assembly/bin.xml b/packaging/src/main/assembly/bin.xml index 6bb4881ed2..58694e8f77 100644 --- a/packaging/src/main/assembly/bin.xml +++ b/packaging/src/main/assembly/bin.xml @@ -88,7 +88,6 @@ org.apache.hive.hcatalog:hive-hcatalog-core org.apache.hive.hcatalog:hive-hcatalog-pig-adapter org.apache.hive.hcatalog:hive-hcatalog-server-extensions - org.apache.hive.hcatalog:hive-hcatalog-streaming