You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2018/05/02 16:44:57 UTC
[1/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Repository: hive
Updated Branches:
refs/heads/master 46c5580b7 -> bf8d305a6
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java b/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java
new file mode 100644
index 0000000..e513915
--- /dev/null
+++ b/streaming/src/test/org/apache/hive/streaming/TestStreamingDynamicPartitioning.java
@@ -0,0 +1,921 @@
+/*
+ * 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.streaming;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+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.List;
+
+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.cli.CliSessionState;
+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.txn.TxnDbUtil;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class TestStreamingDynamicPartitioning {
+ private static final Logger LOG = LoggerFactory.getLogger(TestStreamingDynamicPartitioning.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("Cannot 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 final HiveConf conf;
+ private IDriver driver;
+ private final IMetaStoreClient msClient;
+
+ private static final String COL1 = "id";
+ private static final String COL2 = "msg";
+ @Rule
+ public TemporaryFolder dbFolder = new TemporaryFolder();
+
+ // partitioned table
+ private final static String dbName = "testing";
+ private final static String tblName = "alerts";
+ private final static String[] fieldNames = new String[]{COL1, COL2};
+ private final static String[] colTypes = new String[]{serdeConstants.INT_TYPE_NAME, serdeConstants.STRING_TYPE_NAME};
+ private final static String[] partNames = new String[]{"Continent", "Country"};
+ private final static String[] bucketCols = new String[]{COL1};
+ private final String loc1;
+
+ // unpartitioned table
+ private final static String dbName2 = "testing2";
+
+ public TestStreamingDynamicPartitioning() throws Exception {
+ 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);
+ conf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
+ conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+ conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
+ dbFolder.create();
+ loc1 = dbFolder.newFolder(dbName + ".db").toString();
+
+ //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);
+
+ createDbAndTable(driver, dbName, tblName, null, fieldNames, colTypes, bucketCols, partNames, loc1, 1);
+
+ dropDB(msClient, dbName2);
+ String loc2 = dbFolder.newFolder(dbName2 + ".db").toString();
+ 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 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);
+ }
+
+ @Test
+ public void testDynamicPartitioning() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testing5")
+ .withTable("store_sales")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+
+ for (int i = 0; i < 10; i++) {
+ StringBuilder row = new StringBuilder();
+ 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.append("2018-04-").append(i);
+ connection.write(row.toString().getBytes());
+ }
+ connection.commitTransaction();
+ connection.close();
+
+ List<String> partitions = queryTable(driver, "show partitions testing5.store_sales");
+ // 1 static partition created during setup + 10 dynamic partitions
+ assertEquals(11, partitions.size());
+ // ignore the first static partition
+ for (int i = 1; i < partitions.size(); i++) {
+ assertEquals("dt=2018-04-" + (i - 1), partitions.get(i));
+ }
+
+ ArrayList<String> res = queryTable(driver, "select * from testing5.store_sales");
+ for (String re : res) {
+ System.out.println(re);
+ assertEquals(true, re.contains("2018-04-"));
+ }
+ }
+
+ // 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
+ public void testDPStreamBucketingMatchesRegularBucketing() 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" + "'";
+
+ // disabling vectorization as this test yields incorrect results with vectorization
+ conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false);
+ 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 ) partitioned by (year " +
+ "int) 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 ) partitioned by " +
+ "(year int) location " + tableLoc3);
+ runDDL(driver, "create table finaltable ( bucketid int, key1 string,key2 int,data string ) partitioned " +
+ "by (year int) clustered by ( key1,key2 ) into "
+ + bucketCount + " buckets stored as orc location " + tableLoc2 + " TBLPROPERTIES ('transactional'='true')");
+
+
+ String[] records = new String[]{
+ "PSFAHYLZVC,29,EPNMA,2017",
+ "PPPRKWAYAU,96,VUTEE,2017",
+ "MIAOFERCHI,3,WBDSI,2017",
+ "CEGQAZOWVN,0,WCUZL,2017",
+ "XWAKMNSVQF,28,YJVHU,2017",
+ "XBWTSAJWME,2,KDQFO,2017",
+ "FUVLQTAXAY,5,LDSDG,2017",
+ "QTQMDJMGJH,6,QBOMA,2018",
+ "EFLOTLWJWN,71,GHWPS,2018",
+ "PEQNAOJHCM,82,CAAFI,2018",
+ "MOEKQLGZCP,41,RUACR,2018",
+ "QZXMCOPTID,37,LFLWE,2018",
+ "EYALVWICRD,13,JEZLC,2018",
+ "VYWLZAYTXX,16,DMVZX,2018",
+ "OSALYSQIXR,47,HNZVE,2018",
+ "JGKVHKCEGQ,25,KSCJB,2018",
+ "WQFMMYDHET,12,DTRWA,2018",
+ "AJOVAYZKZQ,15,YBKFO,2018",
+ "YAQONWCUAU,31,QJNHZ,2018",
+ "DJBXUEUOEB,35,IYCBL,2018"
+ };
+
+
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("streamedtable")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+
+ for (String record : records) {
+ connection.write(record.getBytes());
+ }
+
+ connection.commitTransaction();
+ connection.close();
+
+ ArrayList<String> res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2");
+ for (String re : res1) {
+ System.out.println(re);
+ assertTrue(re.endsWith("2017") || re.endsWith("2018"));
+ }
+
+ driver.run("insert into nobucket partition(year) select row__id.bucketid,* from streamedtable");
+ ArrayList<String> res = queryTable(driver, "select * from nobucket");
+ assertEquals(records.length, res.size());
+ runDDL(driver, " insert into finaltable partition(year) select * from nobucket");
+ res = queryTable(driver, "select * from finaltable");
+ assertEquals(records.length, res.size());
+ ArrayList<String> 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());
+
+ res2 = queryTable(driver, "select * from finaltable where year=2018");
+ assertEquals(13, res2.size());
+ for (String s : res2) {
+ assertTrue(s.endsWith("2018"));
+ }
+
+ res2 = queryTable(driver, "show partitions finaltable");
+ assertEquals(2, res2.size());
+ assertEquals("year=2017", res2.get(0));
+ assertEquals("year=2018", res2.get(1));
+ } finally {
+ conf.unset(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED.varname);
+ }
+ }
+
+ @Test
+ public void testDPTwoLevel() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+ connection.write("1,foo,Asia,India".getBytes());
+ connection.write("2,bar,Europe,Germany".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("3,foo,Asia,India".getBytes());
+ connection.write("4,bar,Europe,Germany".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("5,foo,Asia,China".getBytes());
+ connection.write("6,bar,Europe,France".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("7,foo,Asia,China".getBytes());
+ connection.write("8,bar,Europe,France".getBytes());
+ connection.commitTransaction();
+ connection.close();
+
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(8, res.size());
+ assertEquals("1\tfoo\tAsia\tIndia", res.get(0));
+ assertEquals("2\tbar\tEurope\tGermany", res.get(1));
+ assertEquals("3\tfoo\tAsia\tIndia", res.get(2));
+ assertEquals("4\tbar\tEurope\tGermany", res.get(3));
+ assertEquals("5\tfoo\tAsia\tChina", res.get(4));
+ assertEquals("6\tbar\tEurope\tFrance", res.get(5));
+ assertEquals("7\tfoo\tAsia\tChina", res.get(6));
+ assertEquals("8\tbar\tEurope\tFrance", res.get(7));
+
+ res = queryTable(driver, "show partitions " + (dbName + "." + tblName));
+ assertEquals(4, res.size());
+ assertTrue(res.contains("continent=Asia/country=India"));
+ assertTrue(res.contains("continent=Asia/country=China"));
+ assertTrue(res.contains("continent=Europe/country=Germany"));
+ assertTrue(res.contains("continent=Europe/country=France"));
+ }
+
+ @Test
+ public void testDPTwoLevelMissingPartitionValues() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+ connection.write("1,foo,Asia,India".getBytes());
+ connection.write("2,bar,Europe,Germany".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("3,foo,Asia,India".getBytes());
+ connection.write("4,bar,Europe,Germany".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("5,foo,Asia,China".getBytes());
+ connection.write("6,bar,Europe,France".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("7,foo,Asia,China".getBytes());
+ connection.write("8,bar,Europe,France".getBytes());
+ connection.commitTransaction();
+ connection.close();
+
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(8, res.size());
+ assertEquals("1\tfoo\tAsia\tIndia", res.get(0));
+ assertEquals("2\tbar\tEurope\tGermany", res.get(1));
+ assertEquals("3\tfoo\tAsia\tIndia", res.get(2));
+ assertEquals("4\tbar\tEurope\tGermany", res.get(3));
+ assertEquals("5\tfoo\tAsia\tChina", res.get(4));
+ assertEquals("6\tbar\tEurope\tFrance", res.get(5));
+ assertEquals("7\tfoo\tAsia\tChina", res.get(6));
+ assertEquals("8\tbar\tEurope\tFrance", res.get(7));
+
+ res = queryTable(driver, "show partitions " + (dbName + "." + tblName));
+ assertEquals(4, res.size());
+ assertTrue(res.contains("continent=Asia/country=India"));
+ assertTrue(res.contains("continent=Asia/country=China"));
+ assertTrue(res.contains("continent=Europe/country=Germany"));
+ assertTrue(res.contains("continent=Europe/country=France"));
+ }
+
+ @Test
+ public void testDPTwoLevelNonStringPartitionColumns() throws Exception {
+ String tblName = "alerts2";
+ String[] partNames = new String[] {"year", "month"};
+ createDbAndTable(driver, dbName, tblName, null, fieldNames, colTypes, bucketCols, partNames, loc1, 2,
+ "partitioned by (year int, month int)");
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+ connection.write("1,foo,2018,2".getBytes());
+ connection.write("2,bar,2019".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("3,foo,2018".getBytes());
+ connection.write("4,bar,2019".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("5,foo,2018".getBytes());
+ connection.write("6,bar,2019".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("7,foo,,".getBytes());
+ connection.write("8,bar,,12".getBytes());
+ connection.commitTransaction();
+ connection.close();
+
+ // when partition column type is not string, the values from __HIVE_DEFAULT_PARTITION__ will be NULL
+ String defaultPartitionName = "NULL";
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(8, res.size());
+ assertEquals("1\tfoo\t2018\t2", res.get(0));
+ assertEquals("2\tbar\t2019\t" + defaultPartitionName, res.get(1));
+ assertEquals("3\tfoo\t2018\t" + defaultPartitionName, res.get(2));
+ assertEquals("4\tbar\t2019\t" + defaultPartitionName, res.get(3));
+ assertEquals("5\tfoo\t2018\t" + defaultPartitionName, res.get(4));
+ assertEquals("6\tbar\t2019\t" + defaultPartitionName, res.get(5));
+ assertEquals("7\tfoo\t" + defaultPartitionName + "\t" + defaultPartitionName, res.get(6));
+ assertEquals("8\tbar\t" + defaultPartitionName + "\t12", res.get(7));
+
+ defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ res = queryTable(driver, "show partitions " + (dbName + "." + tblName));
+ assertEquals(5, res.size());
+ assertTrue(res.contains("year=2018/month=2"));
+ assertTrue(res.contains("year=2018/month=" + defaultPartitionName));
+ assertTrue(res.contains("year=2019/month=" + defaultPartitionName));
+ assertTrue(res.contains("year=" + defaultPartitionName + "/month=" + defaultPartitionName));
+ assertTrue(res.contains("year=" + defaultPartitionName + "/month=12"));
+ }
+
+ @Test
+ public void testWriteBeforeBegin() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ // begin + write + commit
+ connection.beginTransaction();
+ connection.write("1,foo,Asia".getBytes());
+ connection.write("2,bar,Europe".getBytes());
+ connection.commitTransaction();
+
+ // no begin + write
+ Exception exception = null;
+ try {
+ connection.write("3,SHOULD FAIL!".getBytes());
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().equals("Transaction state is not OPEN. Missing beginTransaction?"));
+
+ // no begin + commit
+ exception = null;
+ try {
+ connection.commitTransaction();
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().equals("Transaction state is not OPEN. Missing beginTransaction?"));
+
+ // no begin + abort
+ exception = null;
+ try {
+ connection.abortTransaction();
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().equals("Transaction state is not OPEN. Missing beginTransaction?"));
+
+ connection.close();
+ String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(2, res.size());
+ assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0));
+ assertEquals("2\tbar\tEurope\t" + defaultPartitionName, res.get(1));
+ }
+
+ @Test
+ public void testWriteAfterClose() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ // begin + write + commit
+ connection.beginTransaction();
+ connection.write("1,foo,Asia".getBytes());
+ connection.write("2,bar,Europe".getBytes());
+ connection.commitTransaction();
+
+ // close + write
+ connection.close();
+
+ Exception exception = null;
+ try {
+ connection.write("3,SHOULD FAIL!".getBytes());
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().endsWith("Streaming connection is closed already."));
+
+ // close + commit
+ exception = null;
+ try {
+ connection.commitTransaction();
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().endsWith("Streaming connection is closed already."));
+
+ // close + abort
+ exception = null;
+ try {
+ connection.abortTransaction();
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().endsWith("Streaming connection is closed already."));
+
+ String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(2, res.size());
+ assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0));
+ assertEquals("2\tbar\tEurope\t" + defaultPartitionName, res.get(1));
+ }
+
+ @Test
+ public void testWriteAfterAbort() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ // begin + write + commit
+ connection.beginTransaction();
+ connection.write("1,foo,Asia".getBytes());
+ connection.write("2,bar,Europe".getBytes());
+ connection.commitTransaction();
+
+ // begin + write + abort
+ connection.beginTransaction();
+ connection.write("3,oops!".getBytes());
+ connection.abortTransaction();
+
+ // begin + write + abort
+ connection.beginTransaction();
+ connection.write("4,I did it again!".getBytes());
+ connection.abortTransaction();
+
+ // begin + write + commit
+ connection.beginTransaction();
+ connection.write("5,Not now!,Europe".getBytes());
+ connection.commitTransaction();
+
+ // close + write
+ connection.close();
+ Exception exception = null;
+ try {
+ connection.write("6,SHOULD FAIL!".getBytes());
+ } catch (Exception e) {
+ exception = e;
+ }
+ assertNotNull(exception);
+ assertTrue(exception.getMessage().equals("Streaming connection is closed already."));
+ String defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ List<String> res = queryTable(driver, "select * from " + (dbName + "." + tblName) + " order by id");
+ assertEquals(3, res.size());
+ assertEquals("1\tfoo\tAsia\t" + defaultPartitionName, res.get(0));
+ assertEquals("2\tbar\tEurope\t" + defaultPartitionName, res.get(1));
+ assertEquals("5\tNot now!\tEurope\t" + defaultPartitionName, res.get(2));
+ }
+
+ @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')");
+
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = null;
+ try {
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("validation2")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+ Assert.assertTrue("InvalidTable exception was not thrown", false);
+ } catch (InvalidTable e) {
+ // expecting this exception
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ try {
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("validation2")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+ Assert.assertTrue("InvalidTable exception was not thrown", false);
+ } catch (InvalidTable e) {
+ // expecting this exception
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private static boolean runDDL(IDriver driver, String sql) {
+ LOG.debug(sql);
+ System.out.println(sql);
+ CommandProcessorResponse cpr = driver.run(sql);
+ if (cpr.getResponseCode() == 0) {
+ return true;
+ }
+ LOG.error("Statement: " + sql + " failed: " + cpr);
+ return false;
+ }
+
+
+ private static ArrayList<String> queryTable(IDriver driver, String query) throws IOException {
+ CommandProcessorResponse cpr = driver.run(query);
+ if (cpr.getResponseCode() != 0) {
+ throw new RuntimeException(query + " failed: " + cpr);
+ }
+ ArrayList<String> res = new ArrayList<String>();
+ driver.getResults(res);
+ return res;
+ }
+
+
+ // 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<String> 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 && partVals != null) {
+ return addPartition(driver, tableName, partVals, partNames);
+ }
+ return new Path(tableLoc);
+ }
+
+ private static Path createDbAndTable(IDriver driver, String databaseName,
+ String tableName, List<String> partVals,
+ String[] colNames, String[] colTypes,
+ String[] bucketCols,
+ String[] partNames, String dbLocation, int bucketCount, String partLine)
+ 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) + " )" +
+ partLine +
+ " clustered by ( " + join(bucketCols, ",") + " )" +
+ " into " + bucketCount + " buckets " +
+ " stored as orc " +
+ " location '" + tableLoc + "'" +
+ " TBLPROPERTIES ('transactional'='true') ";
+ runDDL(driver, crtTbl);
+ if (partNames != null && partNames.length != 0 && partVals != null) {
+ return addPartition(driver, tableName, partVals, partNames);
+ }
+ return new Path(tableLoc);
+ }
+
+ private static Path addPartition(IDriver driver, String tableName, List<String> 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<String> 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 < colNames.length - 1) {
+ sb.append(",");
+ }
+ }
+ return sb.toString();
+ }
+
+ // converts partNames into "partName1 string, partName2 string"
+ private static String getTablePartsStr(String[] partNames) {
+ if (partNames == null || partNames.length == 0) {
+ return "";
+ }
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < partNames.length; ++i) {
+ sb.append(partNames[i]).append(" string");
+ if (i < partNames.length - 1) {
+ sb.append(",");
+ }
+ }
+ return sb.toString();
+ }
+
+ // converts partNames,partVals into "partName1=val1, partName2=val2"
+ private static String getPartsSpec(String[] partNames, List<String> 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) + " )";
+ }
+}
[3/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/InvalidColumn.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/InvalidColumn.java b/streaming/src/java/org/apache/hive/streaming/InvalidColumn.java
deleted file mode 100644
index 0011b14..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-public class InvalidColumn extends StreamingException {
-
- public InvalidColumn(String msg) {
- super(msg);
- }
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/InvalidPartition.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/InvalidPartition.java b/streaming/src/java/org/apache/hive/streaming/InvalidPartition.java
deleted file mode 100644
index f1f9804..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-public class InvalidPartition extends StreamingException {
-
- public InvalidPartition(String partitionName, String partitionValue) {
- super("Invalid partition: Name=" + partitionName +
- ", Value=" + partitionValue);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/InvalidTable.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/InvalidTable.java b/streaming/src/java/org/apache/hive/streaming/InvalidTable.java
index ef1c91d..5c60160 100644
--- a/streaming/src/java/org/apache/hive/streaming/InvalidTable.java
+++ b/streaming/src/java/org/apache/hive/streaming/InvalidTable.java
@@ -24,15 +24,11 @@ public class InvalidTable extends StreamingException {
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) {
+ InvalidTable(String db, String table, String msg) {
super(makeMsg(db, table) + ": " + msg, null);
}
- public InvalidTable(String db, String table, Exception inner) {
+ InvalidTable(String db, String table, Exception inner) {
super(makeMsg(db, table) + ": " + inner.getMessage(), inner);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/InvalidTransactionState.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/InvalidTransactionState.java b/streaming/src/java/org/apache/hive/streaming/InvalidTransactionState.java
new file mode 100644
index 0000000..9d92dfa
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/InvalidTransactionState.java
@@ -0,0 +1,25 @@
+/*
+ * 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.streaming;
+
+public class InvalidTransactionState extends TransactionError {
+ InvalidTransactionState(String msg) {
+ super(msg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/InvalidTrasactionState.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/InvalidTrasactionState.java b/streaming/src/java/org/apache/hive/streaming/InvalidTrasactionState.java
deleted file mode 100644
index 762f5f8..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-public class InvalidTrasactionState extends TransactionError {
- public InvalidTrasactionState(String msg) {
- super(msg);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/PartitionCreationFailed.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/PartitionCreationFailed.java b/streaming/src/java/org/apache/hive/streaming/PartitionCreationFailed.java
index 5f9aca6..e464399 100644
--- a/streaming/src/java/org/apache/hive/streaming/PartitionCreationFailed.java
+++ b/streaming/src/java/org/apache/hive/streaming/PartitionCreationFailed.java
@@ -19,7 +19,7 @@
package org.apache.hive.streaming;
public class PartitionCreationFailed extends StreamingException {
- public PartitionCreationFailed(HiveEndPoint endPoint, Exception cause) {
- super("Failed to create partition " + endPoint, cause);
+ PartitionCreationFailed(StreamingConnection connection, Throwable cause) {
+ super("Failed to create partition " + connection, cause);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/PartitionHandler.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/PartitionHandler.java b/streaming/src/java/org/apache/hive/streaming/PartitionHandler.java
new file mode 100644
index 0000000..1b0e7de
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/PartitionHandler.java
@@ -0,0 +1,33 @@
+/*
+ * 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.streaming;
+
+import java.util.List;
+
+public interface PartitionHandler {
+
+ /**
+ * Creates a partition if it does not exist.
+ *
+ * @param partitionValues - partition values
+ * @return partition location
+ * @throws StreamingException - any metastore related exceptions
+ */
+ PartitionInfo createPartitionIfNotExists(List<String> partitionValues) throws StreamingException;
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/PartitionInfo.java b/streaming/src/java/org/apache/hive/streaming/PartitionInfo.java
new file mode 100644
index 0000000..ce9f76a
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/PartitionInfo.java
@@ -0,0 +1,58 @@
+/*
+ * 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.streaming;
+
+/**
+ * Simple wrapper class for minimal partition related information used by streaming ingest.
+ */
+public class PartitionInfo {
+ private String name;
+ private String partitionLocation;
+ private boolean exists;
+
+ public PartitionInfo(final String name, final String partitionLocation, final boolean exists) {
+ this.name = name;
+ this.partitionLocation = partitionLocation;
+ this.exists = exists;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(final String name) {
+ this.name = name;
+ }
+
+ public String getPartitionLocation() {
+ return partitionLocation;
+ }
+
+ public void setPartitionLocation(final String partitionLocation) {
+ this.partitionLocation = partitionLocation;
+ }
+
+ public boolean isExists() {
+ return exists;
+ }
+
+ public void setExists(final boolean exists) {
+ this.exists = exists;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/QueryFailedException.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/QueryFailedException.java b/streaming/src/java/org/apache/hive/streaming/QueryFailedException.java
deleted file mode 100644
index ccd3ae0..0000000
--- a/streaming/src/java/org/apache/hive/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.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;
- }
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/RecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/RecordWriter.java b/streaming/src/java/org/apache/hive/streaming/RecordWriter.java
index dc6d70e..4d25924 100644
--- a/streaming/src/java/org/apache/hive/streaming/RecordWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/RecordWriter.java
@@ -19,25 +19,44 @@
package org.apache.hive.streaming;
+import java.util.Set;
+
public interface RecordWriter {
- /** Writes using a hive RecordUpdater
+ /**
+ * Initialize record writer.
+ *
+ * @param connection - streaming connection
+ * @param minWriteId - min write id
+ * @param maxWriteID - max write id
+ * @throws StreamingException - thrown when initialization failed
+ */
+ void init(StreamingConnection connection, long minWriteId, long maxWriteID) throws StreamingException;
+
+ /**
+ * 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
+ * @param record the record to be written
*/
void write(long writeId, byte[] record) throws StreamingException;
- /** Flush records from buffer. Invoked by TransactionBatch.commit() */
+ /**
+ * Flush records from buffer. Invoked by TransactionBatch.commitTransaction()
+ */
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()
+ *
+ * @throws StreamingException - thrown when record writer cannot be closed.
+ */
+ void close() throws StreamingException;
- /** Close the RecordUpdater. Invoked by TransactionBatch.close() */
- void closeBatch() throws StreamingException;
+ /**
+ * Get the set of partitions that were added by the record writer.
+ *
+ * @return - set of partitions
+ */
+ Set<String> getPartitions();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/SerializationError.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/SerializationError.java b/streaming/src/java/org/apache/hive/streaming/SerializationError.java
index a57ba00..1473ff8 100644
--- a/streaming/src/java/org/apache/hive/streaming/SerializationError.java
+++ b/streaming/src/java/org/apache/hive/streaming/SerializationError.java
@@ -20,7 +20,7 @@ package org.apache.hive.streaming;
public class SerializationError extends StreamingException {
- public SerializationError(String msg, Exception e) {
+ SerializationError(String msg, Exception e) {
super(msg,e);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StreamingConnection.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StreamingConnection.java b/streaming/src/java/org/apache/hive/streaming/StreamingConnection.java
index 2f760ea..cd7f3d8 100644
--- a/streaming/src/java/org/apache/hive/streaming/StreamingConnection.java
+++ b/streaming/src/java/org/apache/hive/streaming/StreamingConnection.java
@@ -18,40 +18,47 @@
package org.apache.hive.streaming;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.hive.conf.HiveConf;
-/**
- * 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".
- */
-public interface StreamingConnection {
+public interface StreamingConnection extends ConnectionInfo, PartitionHandler {
+ /**
+ * Returns hive configuration object used during connection creation.
+ *
+ * @return - hive conf
+ */
+ HiveConf getHiveConf();
+
+ /**
+ * Begin a transaction for writing.
+ *
+ * @throws StreamingException - if there are errors when beginning transaction
+ */
+ void beginTransaction() throws StreamingException;
+
+ /**
+ * Write record using RecordWriter.
+ *
+ * @param record - the data to be written
+ * @throws StreamingException - if there are errors when writing
+ */
+ void write(byte[] record) throws StreamingException;
/**
- * 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
+ * Commit a transaction to make the writes visible for readers.
+ *
+ * @throws StreamingException - if there are errors when committing the open transaction
*/
- public TransactionBatch fetchTransactionBatch(int numTransactionsHint,
- RecordWriter writer)
- throws ConnectionError, StreamingException, InterruptedException;
+ void commitTransaction() throws StreamingException;
/**
- * Close connection
+ * Manually abort the opened transaction.
+ *
+ * @throws StreamingException - if there are errors when aborting the transaction
*/
- public void close();
+ void abortTransaction() throws StreamingException;
/**
- * @return UserGroupInformation associated with this connection or {@code null} if there is none
+ * Closes streaming connection.
*/
- UserGroupInformation getUserGroupInformation();
+ void close();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StreamingException.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StreamingException.java b/streaming/src/java/org/apache/hive/streaming/StreamingException.java
index a7f84c1..1af5c6a 100644
--- a/streaming/src/java/org/apache/hive/streaming/StreamingException.java
+++ b/streaming/src/java/org/apache/hive/streaming/StreamingException.java
@@ -19,7 +19,7 @@
package org.apache.hive.streaming;
public class StreamingException extends Exception {
- public StreamingException(String msg, Exception cause) {
+ public StreamingException(String msg, Throwable cause) {
super(msg, cause);
}
public StreamingException(String msg) {
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StreamingIOFailure.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StreamingIOFailure.java b/streaming/src/java/org/apache/hive/streaming/StreamingIOFailure.java
index 0dfbfa7..090167d 100644
--- a/streaming/src/java/org/apache/hive/streaming/StreamingIOFailure.java
+++ b/streaming/src/java/org/apache/hive/streaming/StreamingIOFailure.java
@@ -21,11 +21,11 @@ package org.apache.hive.streaming;
public class StreamingIOFailure extends StreamingException {
- public StreamingIOFailure(String msg, Exception cause) {
+ StreamingIOFailure(String msg, Exception cause) {
super(msg, cause);
}
- public StreamingIOFailure(String msg) {
+ StreamingIOFailure(String msg) {
super(msg);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StrictDelimitedInputWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StrictDelimitedInputWriter.java b/streaming/src/java/org/apache/hive/streaming/StrictDelimitedInputWriter.java
new file mode 100644
index 0000000..4a07435
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/StrictDelimitedInputWriter.java
@@ -0,0 +1,110 @@
+/*
+ * 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.streaming;
+
+
+import java.util.Properties;
+
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.serde.serdeConstants;
+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.io.BytesWritable;
+
+import com.google.common.base.Joiner;
+
+/**
+ * Streaming Writer handles delimited input (eg. CSV).
+ * Delimited input is parsed to extract partition values, bucketing info and is forwarded to record updater.
+ * Uses Lazy Simple SerDe to process delimited input.
+ *
+ * NOTE: This record writer is NOT thread-safe. Use one record writer per streaming connection.
+ */
+public class StrictDelimitedInputWriter extends AbstractRecordWriter {
+ private char fieldDelimiter;
+ private char collectionDelimiter;
+ private char mapKeyDelimiter;
+ private LazySimpleSerDe serde;
+
+ private StrictDelimitedInputWriter(Builder builder) {
+ this.fieldDelimiter = builder.fieldDelimiter;
+ this.collectionDelimiter = builder.collectionDelimiter;
+ this.mapKeyDelimiter = builder.mapKeyDelimiter;
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private char fieldDelimiter = (char) LazySerDeParameters.DefaultSeparators[0];
+ private char collectionDelimiter = (char) LazySerDeParameters.DefaultSeparators[1];
+ private char mapKeyDelimiter = (char) LazySerDeParameters.DefaultSeparators[2];
+
+ public Builder withFieldDelimiter(final char fieldDelimiter) {
+ this.fieldDelimiter = fieldDelimiter;
+ return this;
+ }
+
+ public Builder withCollectionDelimiter(final char collectionDelimiter) {
+ this.collectionDelimiter = collectionDelimiter;
+ return this;
+ }
+
+ public Builder withMapKeyDelimiter(final char mapKeyDelimiter) {
+ this.mapKeyDelimiter = mapKeyDelimiter;
+ return this;
+ }
+
+ public StrictDelimitedInputWriter build() {
+ return new StrictDelimitedInputWriter(this);
+ }
+ }
+
+ @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);
+ }
+ }
+
+ @Override
+ public LazySimpleSerDe createSerde() throws SerializationError {
+ try {
+ Properties tableProps = MetaStoreUtils.getTableMetadata(tbl);
+ tableProps.setProperty(serdeConstants.LIST_COLUMNS, Joiner.on(",").join(inputColumns));
+ tableProps.setProperty(serdeConstants.LIST_COLUMN_TYPES, Joiner.on(":").join(inputTypes));
+ tableProps.setProperty(serdeConstants.FIELD_DELIM, String.valueOf(fieldDelimiter));
+ tableProps.setProperty(serdeConstants.COLLECTION_DELIM, String.valueOf(collectionDelimiter));
+ tableProps.setProperty(serdeConstants.MAPKEY_DELIM, String.valueOf(mapKeyDelimiter));
+ LazySimpleSerDe serde = new LazySimpleSerDe();
+ SerDeUtils.initializeSerDe(serde, conf, tableProps, null);
+ this.serde = serde;
+ return serde;
+ } catch (SerDeException e) {
+ throw new SerializationError("Error initializing serde", e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java b/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
index 0077913..1600e7c 100644
--- a/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/StrictJsonWriter.java
@@ -18,131 +18,45 @@
package org.apache.hive.streaming;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Table;
+import java.util.Properties;
+
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.JsonSerDe;
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
+ * Uses org.apache.hadoop.hive.serde2.JsonSerDe to process Json input
+ *
+ * NOTE: This record writer is NOT thread-safe. Use one record writer per streaming connection.
*/
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);
+ public static Builder newBuilder() {
+ return new Builder();
}
- /**
- * @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<? extends StructField> allFields = recordObjInspector.getAllStructFieldRefs();
- for (int i = 0; i < bucketIds.size(); i++) {
- bucketStructFields[i] = allFields.get(bucketIds.get(i));
+ public static class Builder {
+ public StrictJsonWriter build() {
+ return new StrictJsonWriter();
}
}
- @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 {
+ @Override
+ public JsonSerDe createSerde() throws SerializationError {
try {
Properties tableProps = MetaStoreUtils.getTableMetadata(tbl);
JsonSerDe serde = new JsonSerDe();
SerDeUtils.initializeSerDe(serde, conf, tableProps, null);
+ this.serde = serde;
return serde;
} catch (SerDeException e) {
throw new SerializationError("Error initializing serde " + JsonSerDe.class.getName(), e);
@@ -158,5 +72,4 @@ public class StrictJsonWriter extends AbstractRecordWriter {
throw new SerializationError("Unable to convert byte[] record into Object", e);
}
}
-
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/StrictRegexWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/StrictRegexWriter.java b/streaming/src/java/org/apache/hive/streaming/StrictRegexWriter.java
index c0b7324..563cf66 100644
--- a/streaming/src/java/org/apache/hive/streaming/StrictRegexWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/StrictRegexWriter.java
@@ -18,163 +18,73 @@
package org.apache.hive.streaming;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
import java.util.Properties;
import org.apache.commons.lang.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
+ *
+ * NOTE: This record writer is NOT thread-safe. Use one record writer per streaming connection.
*/
public class StrictRegexWriter extends AbstractRecordWriter {
+ private String regex;
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);
+ private StrictRegexWriter(final Builder builder) {
+ this.regex = builder.regex;
}
- /**
- * @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);
+ public static Builder newBuilder() {
+ return new Builder();
}
- /**
- * @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);
- }
+ public static class Builder {
+ private String regex;
- // get StructFields for bucketed cols
- bucketStructFields = new StructField[bucketIds.size()];
- List<? extends StructField> allFields = recordObjInspector.getAllStructFieldRefs();
- for (int i = 0; i < bucketIds.size(); i++) {
- bucketStructFields[i] = allFields.get(bucketIds.get(i));
+ public Builder withRegex(final String regex) {
+ this.regex = regex;
+ return this;
}
- }
-
- @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);
+ public StrictRegexWriter build() {
+ return new StrictRegexWriter(this);
}
}
/**
* 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 {
+ @Override
+ public RegexSerDe createSerde() throws SerializationError {
try {
Properties tableProps = MetaStoreUtils.getTableMetadata(tbl);
tableProps.setProperty(RegexSerDe.INPUT_REGEX, regex);
- ArrayList<String> tableColumns = getCols(tbl);
- tableProps.setProperty(serdeConstants.LIST_COLUMNS, StringUtils.join(tableColumns, ","));
+ tableProps.setProperty(serdeConstants.LIST_COLUMNS, StringUtils.join(inputColumns, ","));
RegexSerDe serde = new RegexSerDe();
SerDeUtils.initializeSerDe(serde, conf, tableProps, null);
+ this.serde = serde;
return serde;
} catch (SerDeException e) {
throw new SerializationError("Error initializing serde " + RegexSerDe.class.getName(), e);
}
}
- private static ArrayList<String> getCols(Table table) {
- List<FieldSchema> cols = table.getSd().getCols();
- ArrayList<String> colNames = new ArrayList<String>(cols.size());
- for (FieldSchema col : cols) {
- colNames.add(col.getName().toLowerCase());
- }
- return colNames;
- }
-
/**
* Encode Utf8 encoded string bytes using RegexSerDe
*
- * @param utf8StrRecord
+ * @param utf8StrRecord - serialized record
* @return The encoded object
- * @throws SerializationError
+ * @throws SerializationError - in case of any deserialization error
*/
@Override
public Object encode(byte[] utf8StrRecord) throws SerializationError {
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/TransactionBatch.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/TransactionBatch.java b/streaming/src/java/org/apache/hive/streaming/TransactionBatch.java
deleted file mode 100644
index 2b05771..0000000
--- a/streaming/src/java/org/apache/hive/streaming/TransactionBatch.java
+++ /dev/null
@@ -1,125 +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.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".
- *
- */
-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<byte[]> 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();
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/TransactionBatchUnAvailable.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/TransactionBatchUnAvailable.java b/streaming/src/java/org/apache/hive/streaming/TransactionBatchUnAvailable.java
deleted file mode 100644
index a8c8cd4..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-public class TransactionBatchUnAvailable extends StreamingException {
- public TransactionBatchUnAvailable(HiveEndPoint ep, Exception e) {
- super("Unable to acquire transaction batch on end point: " + ep, e);
- }
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/TransactionError.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/TransactionError.java b/streaming/src/java/org/apache/hive/streaming/TransactionError.java
index a331b20..ae56e7c 100644
--- a/streaming/src/java/org/apache/hive/streaming/TransactionError.java
+++ b/streaming/src/java/org/apache/hive/streaming/TransactionError.java
@@ -19,11 +19,11 @@
package org.apache.hive.streaming;
public class TransactionError extends StreamingException {
- public TransactionError(String msg, Exception e) {
+ TransactionError(String msg, Exception e) {
super(msg + (e == null ? "" : ": " + e.getMessage()), e);
}
- public TransactionError(String msg) {
+ TransactionError(String msg) {
super(msg);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/test/org/apache/hive/streaming/TestDelimitedInputWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/org/apache/hive/streaming/TestDelimitedInputWriter.java b/streaming/src/test/org/apache/hive/streaming/TestDelimitedInputWriter.java
deleted file mode 100644
index f0843a1..0000000
--- a/streaming/src/test/org/apache/hive/streaming/TestDelimitedInputWriter.java
+++ /dev/null
@@ -1,73 +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.streaming;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-import junit.framework.Assert;
-
-public class TestDelimitedInputWriter {
- @Test
- public void testFieldReordering() throws Exception {
-
- ArrayList<String> 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
- }
- }
- }
-}
[5/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 515c08b..68b4c3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -1038,13 +1038,13 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
}
}
- private static long getHeartbeatInterval(Configuration conf) throws LockException {
+ public static long getHeartbeatInterval(Configuration conf) throws LockException {
// Retrieve HIVE_TXN_TIMEOUT in MILLISECONDS (it's defined as SECONDS),
// then divide it by 2 to give us a safety factor.
long interval =
HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, TimeUnit.MILLISECONDS) / 2;
if (interval == 0) {
- throw new LockException(HiveConf.ConfVars.HIVE_TXN_MANAGER.toString() + " not set," +
+ throw new LockException(HiveConf.ConfVars.HIVE_TXN_TIMEOUT.toString() + " not set," +
" heartbeats won't be sent");
}
return interval;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index f6608eb..5dbc478 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -2378,7 +2378,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
List<org.apache.hadoop.hive.metastore.api.Partition> in =
new ArrayList<org.apache.hadoop.hive.metastore.api.Partition>(size);
for (int i = 0; i < size; ++i) {
- in.add(convertAddSpecToMetaPartition(tbl, addPartitionDesc.getPartition(i)));
+ in.add(convertAddSpecToMetaPartition(tbl, addPartitionDesc.getPartition(i), conf));
}
List<Partition> out = new ArrayList<Partition>();
try {
@@ -2428,8 +2428,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
return out;
}
- private org.apache.hadoop.hive.metastore.api.Partition convertAddSpecToMetaPartition(
- Table tbl, AddPartitionDesc.OnePartitionDesc addSpec) throws HiveException {
+ public static org.apache.hadoop.hive.metastore.api.Partition convertAddSpecToMetaPartition(
+ Table tbl, AddPartitionDesc.OnePartitionDesc addSpec, final HiveConf conf) throws HiveException {
Path location = addSpec.getLocation() != null
? new Path(tbl.getPath(), addSpec.getLocation()) : null;
if (location != null) {
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java
new file mode 100644
index 0000000..40b2e8e
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/JsonSerDe.java
@@ -0,0 +1,669 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hive.serde2;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.charset.CharacterCodingException;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.lazy.LazyFactory;
+import org.apache.hadoop.hive.serde2.lazy.LazyObjectBase;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hive.common.util.HiveStringUtils;
+import org.apache.hive.common.util.TimestampParser;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParseException;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SerDeSpec(schemaProps = {serdeConstants.LIST_COLUMNS,
+ serdeConstants.LIST_COLUMN_TYPES,
+ serdeConstants.TIMESTAMP_FORMATS})
+
+// FIXME: move TestJsonSerDe from hcat to serde2
+public class JsonSerDe extends AbstractSerDe {
+
+ private static final Logger LOG = LoggerFactory.getLogger(JsonSerDe.class);
+ private List<String> columnNames;
+ private StructTypeInfo schema;
+
+ private JsonFactory jsonFactory = null;
+
+ private StandardStructObjectInspector cachedObjectInspector;
+ private TimestampParser tsParser;
+
+ @Override
+ public void initialize(Configuration conf, Properties tbl)
+ throws SerDeException {
+ List<TypeInfo> columnTypes;
+ StructTypeInfo rowTypeInfo;
+
+ LOG.debug("Initializing JsonSerDe: {}", tbl.entrySet());
+
+ // Get column names and types
+ String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS);
+ String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES);
+ final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl
+ .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA);
+ // all table column names
+ if (columnNameProperty.isEmpty()) {
+ columnNames = Collections.emptyList();
+ } else {
+ columnNames = Arrays.asList(columnNameProperty.split(columnNameDelimiter));
+ }
+
+ // all column types
+ if (columnTypeProperty.isEmpty()) {
+ columnTypes = Collections.emptyList();
+ } else {
+ columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty);
+ }
+
+ LOG.debug("columns: {}, {}", columnNameProperty, columnNames);
+ LOG.debug("types: {}, {} ", columnTypeProperty, columnTypes);
+
+ assert (columnNames.size() == columnTypes.size());
+
+ rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes);
+ schema = rowTypeInfo;
+ LOG.debug("schema : {}", schema);
+ cachedObjectInspector = (StandardStructObjectInspector) TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(rowTypeInfo);
+
+ jsonFactory = new JsonFactory();
+ tsParser = new TimestampParser(
+ HiveStringUtils.splitAndUnEscape(tbl.getProperty(serdeConstants.TIMESTAMP_FORMATS)));
+ }
+
+ /**
+ * Takes JSON string in Text form, and has to return an object representation above
+ * it that's readable by the corresponding object inspector.
+ * For this implementation, since we're using the jackson parser, we can construct
+ * our own object implementation, and we use HCatRecord for it
+ */
+ @Override
+ public Object deserialize(Writable blob) throws SerDeException {
+
+ Text t = (Text) blob;
+ JsonParser p;
+ List<Object> r = new ArrayList<>(Collections.nCopies(columnNames.size(), null));
+ try {
+ p = jsonFactory.createJsonParser(new ByteArrayInputStream((t.getBytes())));
+ if (p.nextToken() != JsonToken.START_OBJECT) {
+ throw new IOException("Start token not found where expected");
+ }
+ JsonToken token;
+ while (((token = p.nextToken()) != JsonToken.END_OBJECT) && (token != null)) {
+ // iterate through each token, and create appropriate object here.
+ populateRecord(r, token, p, schema);
+ }
+ } catch (JsonParseException e) {
+ LOG.warn("Error [{}] parsing json text [{}].", e, t);
+ throw new SerDeException(e);
+ } catch (IOException e) {
+ LOG.warn("Error [{}] parsing json text [{}].", e, t);
+ throw new SerDeException(e);
+ }
+
+ return r;
+ }
+
+ private void populateRecord(List<Object> r, JsonToken token, JsonParser p, StructTypeInfo s) throws IOException {
+ if (token != JsonToken.FIELD_NAME) {
+ throw new IOException("Field name expected");
+ }
+ String fieldName = p.getText();
+ int fpos = s.getAllStructFieldNames().indexOf(fieldName);
+ if (fpos == -1) {
+ fpos = getPositionFromHiveInternalColumnName(fieldName);
+ LOG.debug("NPE finding position for field [{}] in schema [{}],"
+ + " attempting to check if it is an internal column name like _col0", fieldName, s);
+ if (fpos == -1) {
+ skipValue(p);
+ return; // unknown field, we return. We'll continue from the next field onwards.
+ }
+ // If we get past this, then the column name did match the hive pattern for an internal
+ // column name, such as _col0, etc, so it *MUST* match the schema for the appropriate column.
+ // This means people can't use arbitrary column names such as _col0, and expect us to ignore it
+ // if we find it.
+ if (!fieldName.equalsIgnoreCase(getHiveInternalColumnName(fpos))) {
+ LOG.error("Hive internal column name {} and position "
+ + "encoding {} for the column name are at odds", fieldName, fpos);
+ throw new IOException("Hive internal column name (" + fieldName
+ + ") and position encoding (" + fpos
+ + ") for the column name are at odds");
+ }
+ // If we reached here, then we were successful at finding an alternate internal
+ // column mapping, and we're about to proceed.
+ }
+ Object currField = extractCurrentField(p, s.getStructFieldTypeInfo(fieldName), false);
+ r.set(fpos, currField);
+ }
+
+ public String getHiveInternalColumnName(int fpos) {
+ return HiveConf.getColumnInternalName(fpos);
+ }
+
+ public int getPositionFromHiveInternalColumnName(String internalName) {
+ // return HiveConf.getPositionFromInternalName(fieldName);
+ // The above line should have been all the implementation that
+ // we need, but due to a bug in that impl which recognizes
+ // only single-digit columns, we need another impl here.
+ Pattern internalPattern = Pattern.compile("_col([0-9]+)");
+ Matcher m = internalPattern.matcher(internalName);
+ if (!m.matches()) {
+ return -1;
+ } else {
+ return Integer.parseInt(m.group(1));
+ }
+ }
+
+ /**
+ * Utility method to extract (and forget) the next value token from the JsonParser,
+ * as a whole. The reason this function gets called is to yank out the next value altogether,
+ * because it corresponds to a field name that we do not recognize, and thus, do not have
+ * a schema/type for. Thus, this field is to be ignored.
+ *
+ * @throws IOException
+ * @throws JsonParseException
+ */
+ private void skipValue(JsonParser p) throws JsonParseException, IOException {
+ JsonToken valueToken = p.nextToken();
+
+ if ((valueToken == JsonToken.START_ARRAY) || (valueToken == JsonToken.START_OBJECT)) {
+ // if the currently read token is a beginning of an array or object, move stream forward
+ // skipping any child tokens till we're at the corresponding END_ARRAY or END_OBJECT token
+ p.skipChildren();
+ }
+ // At the end of this function, the stream should be pointing to the last token that
+ // corresponds to the value being skipped. This way, the next call to nextToken
+ // will advance it to the next field name.
+ }
+
+ /**
+ * Utility method to extract current expected field from given JsonParser
+ * isTokenCurrent is a boolean variable also passed in, which determines
+ * if the JsonParser is already at the token we expect to read next, or
+ * needs advancing to the next before we read.
+ */
+ private Object extractCurrentField(JsonParser p, TypeInfo fieldTypeInfo,
+ boolean isTokenCurrent) throws IOException {
+ Object val = null;
+ JsonToken valueToken;
+ if (isTokenCurrent) {
+ valueToken = p.getCurrentToken();
+ } else {
+ valueToken = p.nextToken();
+ }
+
+ switch (fieldTypeInfo.getCategory()) {
+ case PRIMITIVE:
+ PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = PrimitiveObjectInspector.PrimitiveCategory.UNKNOWN;
+ if (fieldTypeInfo instanceof PrimitiveTypeInfo) {
+ primitiveCategory = ((PrimitiveTypeInfo) fieldTypeInfo).getPrimitiveCategory();
+ }
+ switch (primitiveCategory) {
+ case INT:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getIntValue();
+ break;
+ case BYTE:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getByteValue();
+ break;
+ case SHORT:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getShortValue();
+ break;
+ case LONG:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getLongValue();
+ break;
+ case BOOLEAN:
+ String bval = (valueToken == JsonToken.VALUE_NULL) ? null : p.getText();
+ if (bval != null) {
+ val = Boolean.valueOf(bval);
+ } else {
+ val = null;
+ }
+ break;
+ case FLOAT:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getFloatValue();
+ break;
+ case DOUBLE:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getDoubleValue();
+ break;
+ case STRING:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getText();
+ break;
+ case BINARY:
+ String b = (valueToken == JsonToken.VALUE_NULL) ? null : p.getText();
+ if (b != null) {
+ try {
+ String t = Text.decode(b.getBytes(), 0, b.getBytes().length);
+ return t.getBytes();
+ } catch (CharacterCodingException e) {
+ LOG.warn("Error generating json binary type from object.", e);
+ return null;
+ }
+ } else {
+ val = null;
+ }
+ break;
+ case DATE:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : Date.valueOf(p.getText());
+ break;
+ case TIMESTAMP:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : tsParser.parseTimestamp(p.getText());
+ break;
+ case DECIMAL:
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : HiveDecimal.create(p.getText());
+ break;
+ case VARCHAR:
+ int vLen = ((BaseCharTypeInfo) fieldTypeInfo).getLength();
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : new HiveVarchar(p.getText(), vLen);
+ break;
+ case CHAR:
+ int cLen = ((BaseCharTypeInfo) fieldTypeInfo).getLength();
+ val = (valueToken == JsonToken.VALUE_NULL) ? null : new HiveChar(p.getText(), cLen);
+ break;
+ }
+ break;
+ case LIST:
+ if (valueToken == JsonToken.VALUE_NULL) {
+ val = null;
+ break;
+ }
+ if (valueToken != JsonToken.START_ARRAY) {
+ throw new IOException("Start of Array expected");
+ }
+ List<Object> arr = new ArrayList<Object>();
+ while ((valueToken = p.nextToken()) != JsonToken.END_ARRAY) {
+ arr.add(extractCurrentField(p, ((ListTypeInfo)fieldTypeInfo).getListElementTypeInfo(), true));
+ }
+ val = arr;
+ break;
+ case MAP:
+ if (valueToken == JsonToken.VALUE_NULL) {
+ val = null;
+ break;
+ }
+ if (valueToken != JsonToken.START_OBJECT) {
+ throw new IOException("Start of Object expected");
+ }
+ Map<Object, Object> map = new LinkedHashMap<Object, Object>();
+ while ((valueToken = p.nextToken()) != JsonToken.END_OBJECT) {
+ Object k = getObjectOfCorrespondingPrimitiveType(p.getCurrentName(),
+ (PrimitiveTypeInfo) ((MapTypeInfo)fieldTypeInfo).getMapKeyTypeInfo());
+ Object v = extractCurrentField(p, ((MapTypeInfo) fieldTypeInfo).getMapValueTypeInfo(), false);
+ map.put(k, v);
+ }
+ val = map;
+ break;
+ case STRUCT:
+ if (valueToken == JsonToken.VALUE_NULL) {
+ val = null;
+ break;
+ }
+ if (valueToken != JsonToken.START_OBJECT) {
+ throw new IOException("Start of Object expected");
+ }
+ ArrayList<TypeInfo> subSchema = ((StructTypeInfo)fieldTypeInfo).getAllStructFieldTypeInfos();
+ int sz = subSchema.size();
+ List<Object> struct = new ArrayList<Object>(Collections.nCopies(sz, null));
+ while ((valueToken = p.nextToken()) != JsonToken.END_OBJECT) {
+ populateRecord(struct, valueToken, p, ((StructTypeInfo) fieldTypeInfo));
+ }
+ val = struct;
+ break;
+ default:
+ LOG.error("Unknown type found: " + fieldTypeInfo);
+ return null;
+ }
+ return val;
+ }
+
+ private Object getObjectOfCorrespondingPrimitiveType(String s, PrimitiveTypeInfo mapKeyType)
+ throws IOException {
+ switch (mapKeyType.getPrimitiveCategory()) {
+ case INT:
+ return Integer.valueOf(s);
+ case BYTE:
+ return Byte.valueOf(s);
+ case SHORT:
+ return Short.valueOf(s);
+ case LONG:
+ return Long.valueOf(s);
+ case BOOLEAN:
+ return (s.equalsIgnoreCase("true"));
+ case FLOAT:
+ return Float.valueOf(s);
+ case DOUBLE:
+ return Double.valueOf(s);
+ case STRING:
+ return s;
+ case BINARY:
+ try {
+ String t = Text.decode(s.getBytes(), 0, s.getBytes().length);
+ return t.getBytes();
+ } catch (CharacterCodingException e) {
+ LOG.warn("Error generating json binary type from object.", e);
+ return null;
+ }
+ case DATE:
+ return Date.valueOf(s);
+ case TIMESTAMP:
+ return Timestamp.valueOf(s);
+ case DECIMAL:
+ return HiveDecimal.create(s);
+ case VARCHAR:
+ return new HiveVarchar(s, ((BaseCharTypeInfo) mapKeyType).getLength());
+ case CHAR:
+ return new HiveChar(s, ((BaseCharTypeInfo) mapKeyType).getLength());
+ }
+ throw new IOException("Could not convert from string to map type " + mapKeyType.getTypeName());
+ }
+
+ /**
+ * Given an object and object inspector pair, traverse the object
+ * and generate a Text representation of the object.
+ */
+ @Override
+ public Writable serialize(Object obj, ObjectInspector objInspector)
+ throws SerDeException {
+ StringBuilder sb = new StringBuilder();
+ try {
+
+ StructObjectInspector soi = (StructObjectInspector) objInspector;
+ List<? extends StructField> structFields = soi.getAllStructFieldRefs();
+ assert (columnNames.size() == structFields.size());
+ if (obj == null) {
+ sb.append("null");
+ } else {
+ sb.append(SerDeUtils.LBRACE);
+ for (int i = 0; i < structFields.size(); i++) {
+ if (i > 0) {
+ sb.append(SerDeUtils.COMMA);
+ }
+ appendWithQuotes(sb, columnNames.get(i));
+ sb.append(SerDeUtils.COLON);
+ buildJSONString(sb, soi.getStructFieldData(obj, structFields.get(i)),
+ structFields.get(i).getFieldObjectInspector());
+ }
+ sb.append(SerDeUtils.RBRACE);
+ }
+
+ } catch (IOException e) {
+ LOG.warn("Error generating json text from object.", e);
+ throw new SerDeException(e);
+ }
+ return new Text(sb.toString());
+ }
+
+ private static StringBuilder appendWithQuotes(StringBuilder sb, String value) {
+ return sb == null ? null : sb.append(SerDeUtils.QUOTE).append(value).append(SerDeUtils.QUOTE);
+ }
+
+ // TODO : code section copied over from SerDeUtils because of non-standard json production there
+ // should use quotes for all field names. We should fix this there, and then remove this copy.
+ // See http://jackson.codehaus.org/1.7.3/javadoc/org/codehaus/jackson/JsonParser.Feature.html#ALLOW_UNQUOTED_FIELD_NAMES
+ // for details - trying to enable Jackson to ignore that doesn't seem to work(compilation failure
+ // when attempting to use that feature, so having to change the production itself.
+ // Also, throws IOException when Binary is detected.
+ private static void buildJSONString(StringBuilder sb, Object o, ObjectInspector oi) throws IOException {
+
+ switch (oi.getCategory()) {
+ case PRIMITIVE: {
+ PrimitiveObjectInspector poi = (PrimitiveObjectInspector) oi;
+ if (o == null) {
+ sb.append("null");
+ } else {
+ switch (poi.getPrimitiveCategory()) {
+ case BOOLEAN: {
+ boolean b = ((BooleanObjectInspector) poi).get(o);
+ sb.append(b ? "true" : "false");
+ break;
+ }
+ case BYTE: {
+ sb.append(((ByteObjectInspector) poi).get(o));
+ break;
+ }
+ case SHORT: {
+ sb.append(((ShortObjectInspector) poi).get(o));
+ break;
+ }
+ case INT: {
+ sb.append(((IntObjectInspector) poi).get(o));
+ break;
+ }
+ case LONG: {
+ sb.append(((LongObjectInspector) poi).get(o));
+ break;
+ }
+ case FLOAT: {
+ sb.append(((FloatObjectInspector) poi).get(o));
+ break;
+ }
+ case DOUBLE: {
+ sb.append(((DoubleObjectInspector) poi).get(o));
+ break;
+ }
+ case STRING: {
+ String s =
+ SerDeUtils.escapeString(((StringObjectInspector) poi).getPrimitiveJavaObject(o));
+ appendWithQuotes(sb, s);
+ break;
+ }
+ case BINARY:
+ byte[] b = ((BinaryObjectInspector) oi).getPrimitiveJavaObject(o);
+ Text txt = new Text();
+ txt.set(b, 0, b.length);
+ appendWithQuotes(sb, SerDeUtils.escapeString(txt.toString()));
+ break;
+ case DATE:
+ Date d = ((DateObjectInspector) poi).getPrimitiveJavaObject(o);
+ appendWithQuotes(sb, d.toString());
+ break;
+ case TIMESTAMP: {
+ Timestamp t = ((TimestampObjectInspector) poi).getPrimitiveJavaObject(o);
+ appendWithQuotes(sb, t.toString());
+ break;
+ }
+ case DECIMAL:
+ sb.append(((HiveDecimalObjectInspector) poi).getPrimitiveJavaObject(o));
+ break;
+ case VARCHAR: {
+ String s = SerDeUtils.escapeString(
+ ((HiveVarcharObjectInspector) poi).getPrimitiveJavaObject(o).toString());
+ appendWithQuotes(sb, s);
+ break;
+ }
+ case CHAR: {
+ //this should use HiveChar.getPaddedValue() but it's protected; currently (v0.13)
+ // HiveChar.toString() returns getPaddedValue()
+ String s = SerDeUtils.escapeString(
+ ((HiveCharObjectInspector) poi).getPrimitiveJavaObject(o).toString());
+ appendWithQuotes(sb, s);
+ break;
+ }
+ default:
+ throw new RuntimeException("Unknown primitive type: " + poi.getPrimitiveCategory());
+ }
+ }
+ break;
+ }
+ case LIST: {
+ ListObjectInspector loi = (ListObjectInspector) oi;
+ ObjectInspector listElementObjectInspector = loi
+ .getListElementObjectInspector();
+ List<?> olist = loi.getList(o);
+ if (olist == null) {
+ sb.append("null");
+ } else {
+ sb.append(SerDeUtils.LBRACKET);
+ for (int i = 0; i < olist.size(); i++) {
+ if (i > 0) {
+ sb.append(SerDeUtils.COMMA);
+ }
+ buildJSONString(sb, olist.get(i), listElementObjectInspector);
+ }
+ sb.append(SerDeUtils.RBRACKET);
+ }
+ break;
+ }
+ case MAP: {
+ MapObjectInspector moi = (MapObjectInspector) oi;
+ ObjectInspector mapKeyObjectInspector = moi.getMapKeyObjectInspector();
+ ObjectInspector mapValueObjectInspector = moi
+ .getMapValueObjectInspector();
+ Map<?, ?> omap = moi.getMap(o);
+ if (omap == null) {
+ sb.append("null");
+ } else {
+ sb.append(SerDeUtils.LBRACE);
+ boolean first = true;
+ for (Object entry : omap.entrySet()) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(SerDeUtils.COMMA);
+ }
+ Map.Entry<?, ?> e = (Map.Entry<?, ?>) entry;
+ StringBuilder keyBuilder = new StringBuilder();
+ buildJSONString(keyBuilder, e.getKey(), mapKeyObjectInspector);
+ String keyString = keyBuilder.toString().trim();
+ if ((!keyString.isEmpty()) && (keyString.charAt(0) != SerDeUtils.QUOTE)) {
+ appendWithQuotes(sb, keyString);
+ } else {
+ sb.append(keyString);
+ }
+ sb.append(SerDeUtils.COLON);
+ buildJSONString(sb, e.getValue(), mapValueObjectInspector);
+ }
+ sb.append(SerDeUtils.RBRACE);
+ }
+ break;
+ }
+ case STRUCT: {
+ StructObjectInspector soi = (StructObjectInspector) oi;
+ List<? extends StructField> structFields = soi.getAllStructFieldRefs();
+ if (o == null) {
+ sb.append("null");
+ } else {
+ sb.append(SerDeUtils.LBRACE);
+ for (int i = 0; i < structFields.size(); i++) {
+ if (i > 0) {
+ sb.append(SerDeUtils.COMMA);
+ }
+ appendWithQuotes(sb, structFields.get(i).getFieldName());
+ sb.append(SerDeUtils.COLON);
+ buildJSONString(sb, soi.getStructFieldData(o, structFields.get(i)),
+ structFields.get(i).getFieldObjectInspector());
+ }
+ sb.append(SerDeUtils.RBRACE);
+ }
+ break;
+ }
+ case UNION: {
+ UnionObjectInspector uoi = (UnionObjectInspector) oi;
+ if (o == null) {
+ sb.append("null");
+ } else {
+ sb.append(SerDeUtils.LBRACE);
+ sb.append(uoi.getTag(o));
+ sb.append(SerDeUtils.COLON);
+ buildJSONString(sb, uoi.getField(o),
+ uoi.getObjectInspectors().get(uoi.getTag(o)));
+ sb.append(SerDeUtils.RBRACE);
+ }
+ break;
+ }
+ default:
+ throw new RuntimeException("Unknown type in ObjectInspector!");
+ }
+ }
+
+
+ /**
+ * Returns an object inspector for the specified schema that
+ * is capable of reading in the object representation of the JSON string
+ */
+ @Override
+ public ObjectInspector getObjectInspector() throws SerDeException {
+ return cachedObjectInspector;
+ }
+
+ @Override
+ public Class<? extends Writable> getSerializedClass() {
+ return Text.class;
+ }
+
+ @Override
+ public SerDeStats getSerDeStats() {
+ // no support for statistics yet
+ return null;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index 8c159e9..742b6bf 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore.utils;
import org.apache.hadoop.hive.metastore.api.WMPoolSchedulingPolicy;
+import com.google.common.base.Joiner;
import com.google.common.base.Predicates;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
@@ -90,6 +91,7 @@ import java.util.Properties;
import java.util.Map.Entry;
import java.util.SortedMap;
import java.util.SortedSet;
+import java.util.StringJoiner;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.Callable;
@@ -1264,7 +1266,7 @@ public class MetaStoreUtils {
if (sd.getBucketCols() != null && sd.getBucketCols().size() > 0) {
schema.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_FIELD_NAME,
- sd.getBucketCols().get(0));
+ Joiner.on(",").join(sd.getBucketCols()));
}
// SerdeInfo
@@ -1371,8 +1373,8 @@ public class MetaStoreUtils {
.toString(sd.getNumBuckets()));
if (sd.getBucketCols() != null && sd.getBucketCols().size() > 0) {
schema.setProperty(
- org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_FIELD_NAME, sd
- .getBucketCols().get(0));
+ org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.BUCKET_FIELD_NAME,
+ Joiner.on(",").join(sd.getBucketCols()));
}
if (sd.getSerdeInfo() != null) {
for (Map.Entry<String,String> param : sd.getSerdeInfo().getParameters().entrySet()) {
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index b58ec01..ccc55eb 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -33,7 +33,7 @@
</properties>
<dependencies>
- <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+ <!-- dependencies are always listed in sorted order by groupId, artifactId -->
<!-- intra-project -->
<dependency>
<groupId>org.apache.hive</groupId>
@@ -51,21 +51,9 @@
<version>${project.version}</version>
</dependency>
<dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-cli</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hive.hcatalog</groupId>
- <artifactId>hive-hcatalog-core</artifactId>
- <optional>true</optional>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-lang3</artifactId>
- <optional>true</optional>
- <version>3.3.2</version>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ <version>${slf4j.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -78,8 +66,8 @@
<artifactId>hadoop-mapreduce-client-core</artifactId>
<optional>true</optional>
<version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
+ <exclusions>
+ <exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
@@ -97,24 +85,12 @@
<version>${junit.version}</version>
<scope>test</scope>
</dependency>
-
<dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-mapreduce-client-common</artifactId>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-cli</artifactId>
+ <version>${project.version}</version>
<scope>test</scope>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
- </exclusion>
- <exclusion>
- <groupId>commmons-logging</groupId>
- <artifactId>commons-logging</artifactId>
- </exclusion>
- </exclusions>
</dependency>
-
</dependencies>
<build>
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
index 25998ae..643bcc4 100644
--- a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
@@ -19,17 +19,28 @@
package org.apache.hive.streaming;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
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.HiveMetaStoreUtils;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
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.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
import org.apache.hadoop.hive.ql.io.RecordUpdater;
import org.apache.hadoop.hive.serde2.AbstractSerDe;
@@ -38,162 +49,202 @@ 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.hive.serde2.objectinspector.SubStructObjectInspector;
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;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
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<Integer> bucketIds;
- private ArrayList<RecordUpdater> updaters = null;
-
- private final int totalBuckets;
- /**
- * Indicates whether target table is bucketed
- */
- private final boolean isBucketed;
+ private static final Logger LOG = LoggerFactory.getLogger(AbstractRecordWriter.class.getName());
- private final Path partitionPath;
-
- private final AcidOutputFormat<?,?> outf;
- private Object[] bucketFieldData; // Pre-allocated in constructor. Updated on each write.
+ protected HiveConf conf;
+ private StreamingConnection conn;
+ protected Table tbl;
+ List<String> inputColumns;
+ List<String> inputTypes;
+ private String fullyQualifiedTableName;
+ private Map<String, List<RecordUpdater>> updaters = new HashMap<>();
+ private Map<String, Path> partitionPaths = new HashMap<>();
+ private Set<String> addedPartitions = new HashSet<>();
+ // input OI includes table columns + partition columns
+ private StructObjectInspector inputRowObjectInspector;
+ // output OI strips off the partition columns and retains other columns
+ private ObjectInspector outputRowObjectInspector;
+ private List<String> partitionColumns = new ArrayList<>();
+ private ObjectInspector[] partitionObjInspectors = null;
+ private StructField[] partitionStructFields = null;
+ private Object[] partitionFieldData;
+ private ObjectInspector[] bucketObjInspectors = null;
+ private StructField[] bucketStructFields = null;
+ private Object[] bucketFieldData;
+ private List<Integer> bucketIds = new ArrayList<>();
+ private int totalBuckets;
+ private String defaultPartitionName;
+ private boolean isBucketed;
+ private AcidOutputFormat<?, ?> acidOutputFormat;
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;
+ @Override
+ public void init(StreamingConnection conn, long minWriteId, long maxWriteId) throws StreamingException {
+ if (conn == null) {
+ throw new StreamingException("Streaming connection cannot be null during record writer initialization");
}
- }
- /**
- * @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<TableWriterPair>() {
- @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();
+ this.conn = conn;
+ this.curBatchMinWriteId = minWriteId;
+ this.curBatchMaxWriteId = maxWriteId;
+ this.conf = conn.getHiveConf();
+ this.defaultPartitionName = conf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ final IMetaStoreClient msClient = HiveMetaStoreUtils.getHiveMetastoreClient(this.conf);
+ this.tbl = msClient.getTable(conn.getDatabase(), conn.getTable());
+ this.inputColumns = tbl.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList());
+ this.inputTypes = tbl.getSd().getCols().stream().map(FieldSchema::getType).collect(Collectors.toList());
+ if (conn.isPartitionedTable() && conn.isDynamicPartitioning()) {
+ this.partitionColumns = tbl.getPartitionKeys().stream().map(FieldSchema::getName).collect(Collectors.toList());
+ this.inputColumns.addAll(partitionColumns);
+ this.inputTypes.addAll(tbl.getPartitionKeys().stream().map(FieldSchema::getType).collect(Collectors.toList()));
}
+ this.fullyQualifiedTableName = Warehouse.getQualifiedName(conn.getDatabase(), conn.getTable());
String outFormatName = this.tbl.getSd().getOutputFormat();
- outf = (AcidOutputFormat<?, ?>) ReflectionUtils.newInstance(JavaUtils.loadClass(outFormatName), conf);
- } catch(InterruptedException e) {
- throw new StreamingException(endPoint2.toString(), e);
+ this.acidOutputFormat = (AcidOutputFormat<?, ?>) ReflectionUtils.newInstance(JavaUtils.loadClass(outFormatName), conf);
} catch (MetaException | NoSuchObjectException e) {
- throw new ConnectionError(endPoint2, e);
+ throw new ConnectionError(conn, e);
} catch (TException | ClassNotFoundException | IOException e) {
throw new StreamingException(e.getMessage(), e);
}
+
+ try {
+ final AbstractSerDe serDe = createSerde();
+ this.inputRowObjectInspector = (StructObjectInspector) serDe.getObjectInspector();
+ if (conn.isPartitionedTable() && conn.isDynamicPartitioning()) {
+ preparePartitioningFields();
+ int dpStartCol = inputRowObjectInspector.getAllStructFieldRefs().size() - tbl.getPartitionKeys().size();
+ this.outputRowObjectInspector = new SubStructObjectInspector(inputRowObjectInspector, 0, dpStartCol);
+ } else {
+ this.outputRowObjectInspector = inputRowObjectInspector;
+ }
+ prepareBucketingFields();
+ } catch (SerDeException e) {
+ throw new StreamingException("Unable to create SerDe", e);
+ }
+ }
+
+ private void prepareBucketingFields() {
+ this.isBucketed = tbl.getSd().getNumBuckets() > 0;
+ // For unbucketed tables we have exactly 1 RecordUpdater (until HIVE-19208) 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()];
+ this.bucketObjInspectors = getObjectInspectorsForBucketedCols(bucketIds, inputRowObjectInspector);
+ this.bucketStructFields = new StructField[bucketIds.size()];
+ List<? extends StructField> allFields = inputRowObjectInspector.getAllStructFieldRefs();
+ for (int i = 0; i < bucketIds.size(); i++) {
+ bucketStructFields[i] = allFields.get(bucketIds.get(i));
+ }
+ }
+ }
+
+ private void preparePartitioningFields() {
+ final int numPartitions = tbl.getPartitionKeys().size();
+ this.partitionFieldData = new Object[numPartitions];
+ this.partitionObjInspectors = new ObjectInspector[numPartitions];
+ int startIdx = inputRowObjectInspector.getAllStructFieldRefs().size() - numPartitions;
+ int endIdx = inputRowObjectInspector.getAllStructFieldRefs().size();
+ int j = 0;
+ for (int i = startIdx; i < endIdx; i++) {
+ StructField structField = inputRowObjectInspector.getAllStructFieldRefs().get(i);
+ partitionObjInspectors[j++] = structField.getFieldObjectInspector();
+ }
+ this.partitionStructFields = new StructField[partitionColumns.size()];
+ for (int i = 0; i < partitionColumns.size(); i++) {
+ String partCol = partitionColumns.get(i);
+ partitionStructFields[i] = inputRowObjectInspector.getStructFieldRef(partCol);
+ }
}
/**
- * used to tag error msgs to provied some breadcrumbs
+ * used to tag error msgs to provided some breadcrumbs
*/
- String getWatermark() {
- return partitionPath + " writeIds[" + curBatchMinWriteId + "," + curBatchMaxWriteId + "]";
+ private String getWatermark(String partition) {
+ return partition + " writeIds[" + curBatchMinWriteId + "," + curBatchMaxWriteId + "]";
}
+
// return the column numbers of the bucketed columns
private List<Integer> getBucketColIDs(List<String> bucketCols, List<FieldSchema> cols) {
- ArrayList<Integer> result = new ArrayList<Integer>(bucketCols.size());
- HashSet<String> bucketSet = new HashSet<String>(bucketCols);
+ ArrayList<Integer> result = new ArrayList<>(bucketCols.size());
+ HashSet<String> bucketSet = new HashSet<>(bucketCols);
for (int i = 0; i < cols.size(); i++) {
- if( bucketSet.contains(cols.get(i).getName()) ) {
+ 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;
+ public abstract AbstractSerDe createSerde() 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.
+ * serde returned by {@link #createSerde}. This is public so that test frameworks can use it.
+ *
* @param record record to be deserialized
* @return deserialized record as an Object
- * @throws SerializationError
+ * @throws SerializationError - any error during serialization or deserialization of record
*/
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) {
+ private int getBucket(Object row) {
+ if (!isBucketed) {
return 0;
}
- ObjectInspector[] inspectors = getBucketObjectInspectors();
Object[] bucketFields = getBucketFields(row);
- return ObjectInspectorUtils.getBucketNumber(bucketFields, inspectors, totalBuckets);
+ int bucketingVersion = Utilities.getBucketingVersion(
+ tbl.getParameters().get(hive_metastoreConstants.TABLE_BUCKETING_VERSION));
+
+ return bucketingVersion == 2 ?
+ ObjectInspectorUtils.getBucketNumber(bucketFields, bucketObjInspectors, totalBuckets) :
+ ObjectInspectorUtils.getBucketNumberOld(bucketFields, bucketObjInspectors, totalBuckets);
+ }
+
+ private List<String> getPartitionValues(final Object row) {
+ if (!conn.isPartitionedTable()) {
+ return null;
+ }
+ List<String> partitionValues = new ArrayList<>();
+ if (conn.isPartitionedTable() && conn.isDynamicPartitioning()) {
+ Object[] partitionFields = getPartitionFields(row);
+ for (int i = 0; i < partitionObjInspectors.length; i++) {
+ ObjectInspector oi = partitionObjInspectors[i];
+ Object field = partitionFields[i];
+ Object partitionValue = ObjectInspectorUtils.copyToStandardObject(field, oi, ObjectInspectorUtils
+ .ObjectInspectorCopyOption.WRITABLE);
+ if (partitionValue == null || partitionValue.toString().length() == 0) {
+ partitionValues.add(defaultPartitionName);
+ } else {
+ partitionValues.add(partitionValue.toString());
+ }
+ }
+ } else {
+ partitionValues = conn.getStaticPartitionValues();
+ }
+ return partitionValues;
}
@Override
public void flush() throws StreamingIOFailure {
try {
- for (RecordUpdater updater : updaters) {
- if (updater != null) {
- updater.flush();
+ for (Map.Entry<String, List<RecordUpdater>> entry : updaters.entrySet()) {
+ LOG.info("Flushing record updater for partitions: {}", entry.getKey());
+ for (RecordUpdater updater : entry.getValue()) {
+ if (updater != null) {
+ updater.flush();
+ }
}
}
} catch (IOException e) {
@@ -202,123 +253,146 @@ public abstract class AbstractRecordWriter implements RecordWriter {
}
@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<RecordUpdater>(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 {
+ public void close() 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);
+ String partition = null;
+ for (Map.Entry<String, List<RecordUpdater>> entry : updaters.entrySet()) {
+ partition = entry.getKey();
+ LOG.info("Closing updater for partitions: {}", partition);
+ for (RecordUpdater updater : entry.getValue()) {
+ 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);
+ }
}
}
+ entry.getValue().clear();
}
updaters.clear();
- if(haveError) {
- throw new StreamingIOFailure("Encountered errors while closing (see logs) " + getWatermark());
+ if (haveError) {
+ throw new StreamingIOFailure("Encountered errors while closing (see logs) " + getWatermark(partition));
}
}
- protected static ObjectInspector[] getObjectInspectorsForBucketedCols(List<Integer> bucketIds
- , StructObjectInspector recordObjInspector)
- throws SerializationError {
+ private static ObjectInspector[] getObjectInspectorsForBucketedCols(List<Integer> bucketIds
+ , StructObjectInspector recordObjInspector) {
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();
+ recordObjInspector.getAllStructFieldRefs().get(bucketId).getFieldObjectInspector();
}
return result;
}
-
- private Object[] getBucketFields(Object row) throws SerializationError {
- StructObjectInspector recordObjInspector = getRecordObjectInspector();
- StructField[] bucketStructFields = getBucketStructFields();
+ private Object[] getBucketFields(Object row) {
for (int i = 0; i < bucketIds.size(); i++) {
- bucketFieldData[i] = recordObjInspector.getStructFieldData(row, bucketStructFields[i]);
+ bucketFieldData[i] = inputRowObjectInspector.getStructFieldData(row, bucketStructFields[i]);
}
return bucketFieldData;
}
- private RecordUpdater createRecordUpdater(int bucketId, Long minWriteId, Long maxWriteID)
- throws IOException, SerializationError {
+ private Object[] getPartitionFields(Object row) {
+ for (int i = 0; i < partitionFieldData.length; i++) {
+ partitionFieldData[i] = inputRowObjectInspector.getStructFieldData(row, partitionStructFields[i]);
+ }
+ return partitionFieldData;
+ }
+
+ @Override
+ public void write(final long writeId, final byte[] record) throws StreamingException {
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);
+ Object encodedRow = encode(record);
+ int bucket = getBucket(encodedRow);
+ List<String> partitionValues = getPartitionValues(encodedRow);
+ getRecordUpdater(partitionValues, bucket).insert(writeId, encodedRow);
+ } catch (IOException e) {
+ throw new StreamingIOFailure("Error writing record in transaction write id ("
+ + writeId + ")", e);
}
}
- RecordUpdater getRecordUpdater(int bucketId) throws StreamingIOFailure, SerializationError {
- RecordUpdater recordUpdater = updaters.get(bucketId);
+ @Override
+ public Set<String> getPartitions() {
+ return addedPartitions;
+ }
+
+ private RecordUpdater createRecordUpdater(final Path partitionPath, int bucketId, Long minWriteId,
+ Long maxWriteID)
+ throws IOException {
+ // 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 acidOutputFormat.getRecordUpdater(partitionPath,
+ new AcidOutputFormat.Options(conf)
+ .inspector(outputRowObjectInspector)
+ .bucket(bucketId)
+ .tableProperties(tblProperties)
+ .minimumWriteId(minWriteId)
+ .maximumWriteId(maxWriteID)
+ .statementId(-1)
+ .finalDestination(partitionPath));
+ }
+
+ private RecordUpdater getRecordUpdater(List<String> partitionValues, int bucketId) throws StreamingIOFailure {
+ RecordUpdater recordUpdater;
+ String key;
+ Path destLocation;
+ try {
+ key = partitionValues == null ? fullyQualifiedTableName : partitionValues.toString();
+ // add partition in metastore for dynamic partition. We make a metastore call for every new partition value that
+ // we encounter even if partition already exists (exists check require a metastore call anyways).
+ if (partitionPaths.containsKey(key)) {
+ destLocation = partitionPaths.get(key);
+ } else {
+ // un-partitioned table
+ if (partitionValues == null) {
+ destLocation = new Path(tbl.getSd().getLocation());
+ } else {
+ PartitionInfo partitionInfo = conn.createPartitionIfNotExists(partitionValues);
+ // collect the newly added partitions. connection.commitTransaction() will report the dynamically added
+ // partitions to TxnHandler
+ if (!partitionInfo.isExists()) {
+ addedPartitions.add(partitionInfo.getName());
+ LOG.info("Created partition {} for table {}", partitionInfo.getName(), fullyQualifiedTableName);
+ } else {
+ LOG.info("Partition {} already exists for table {}", partitionInfo.getName(), fullyQualifiedTableName);
+ }
+ destLocation = new Path(partitionInfo.getPartitionLocation());
+ }
+ partitionPaths.put(key, destLocation);
+ }
+ updaters.computeIfAbsent(key, k -> initializeBuckets());
+ recordUpdater = updaters.get(key).get(bucketId);
+ } catch (StreamingException e) {
+ throw new StreamingIOFailure("Unable to create partition: " + partitionValues + "for " + conn, e);
+ }
if (recordUpdater == null) {
try {
- recordUpdater = createRecordUpdater(bucketId, curBatchMinWriteId, curBatchMaxWriteId);
+ recordUpdater = createRecordUpdater(destLocation, bucketId, curBatchMinWriteId, curBatchMaxWriteId);
} catch (IOException e) {
- String errMsg = "Failed creating RecordUpdater for " + getWatermark();
+ String errMsg = "Failed creating RecordUpdater for " + getWatermark(destLocation.toString());
LOG.error(errMsg, e);
throw new StreamingIOFailure(errMsg, e);
}
- updaters.set(bucketId, recordUpdater);
+ List<RecordUpdater> partitionUpdaters = updaters.get(key);
+ partitionUpdaters.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);
+ private List<RecordUpdater> initializeBuckets() {
+ List<RecordUpdater> result = new ArrayList<>(totalBuckets);
+ for (int bucket = 0; bucket < totalBuckets; bucket++) {
+ result.add(bucket, null); //so that get(i) returns null rather than ArrayOutOfBounds
}
+ return result;
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/ConnectionError.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/ConnectionError.java b/streaming/src/java/org/apache/hive/streaming/ConnectionError.java
index 668bffb..9b6a6aa 100644
--- a/streaming/src/java/org/apache/hive/streaming/ConnectionError.java
+++ b/streaming/src/java/org/apache/hive/streaming/ConnectionError.java
@@ -20,15 +20,15 @@ package org.apache.hive.streaming;
public class ConnectionError extends StreamingException {
- public ConnectionError(String msg) {
+ ConnectionError(String msg) {
super(msg);
}
- public ConnectionError(String msg, Exception innerEx) {
+ ConnectionError(String msg, Exception innerEx) {
super(msg, innerEx);
}
- public ConnectionError(HiveEndPoint endPoint, Exception innerEx) {
+ ConnectionError(StreamingConnection endPoint, Exception innerEx) {
super("Error connecting to " + endPoint +
(innerEx == null ? "" : ": " + innerEx.getMessage()), innerEx);
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/ConnectionInfo.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/ConnectionInfo.java b/streaming/src/java/org/apache/hive/streaming/ConnectionInfo.java
new file mode 100644
index 0000000..ca8babf
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/ConnectionInfo.java
@@ -0,0 +1,76 @@
+/*
+ * 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.streaming;
+
+import java.util.List;
+
+/**
+ * Helper interface to get connection related information.
+ */
+public interface ConnectionInfo {
+
+ /**
+ * Get metastore URI that metastore client uses.
+ *
+ * @return - metastore URI used by client
+ */
+ String getMetastoreUri();
+
+ /**
+ * Get the database used by streaming connection.
+ *
+ * @return - database
+ */
+ String getDatabase();
+
+ /**
+ * Get the table used by streaming connection.
+ *
+ * @return - table
+ */
+ String getTable();
+
+ /**
+ * Get any static partitions specified during streaming connection creation.
+ *
+ * @return - static partition values
+ */
+ List<String> getStaticPartitionValues();
+
+ /**
+ * Get if the specified table is partitioned table or not.
+ *
+ * @return - true if partitioned table else false
+ */
+ boolean isPartitionedTable();
+
+ /**
+ * Get if dynamic partitioning is used.
+ *
+ * @return - true if dynamic partitioning case else false
+ */
+ boolean isDynamicPartitioning();
+
+ /**
+ * Get agent info that is set during streaming connection.
+ *
+ * @return - agent info
+ */
+ String getAgentInfo();
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/DelimitedInputWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/DelimitedInputWriter.java b/streaming/src/java/org/apache/hive/streaming/DelimitedInputWriter.java
deleted file mode 100644
index 898b3f9..0000000
--- a/streaming/src/java/org/apache/hive/streaming/DelimitedInputWriter.java
+++ /dev/null
@@ -1,331 +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.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
- */
-public class DelimitedInputWriter extends AbstractRecordWriter {
- private final boolean reorderingNeeded;
- private String delimiter;
- private char serdeSeparator;
- private int[] fieldToColMapping;
- private final ArrayList<String> 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<? extends StructField> 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<String> tableColumns) {
- return !( delimiter.equals(String.valueOf(getSerdeSeparator()))
- && areFieldsInColOrder(fieldToColMapping)
- && tableColumns.size()>=fieldToColMapping.length );
- }
-
- private static boolean areFieldsInColOrder(int[] fieldToColMapping) {
- for(int i=0; i<fieldToColMapping.length; ++i) {
- if(fieldToColMapping[i]!=i) {
- return false;
- }
- }
- return true;
- }
-
- @VisibleForTesting
- static int[] getFieldReordering(String[] colNamesForFields, List<String> tableColNames)
- throws InvalidColumn {
- int[] result = new int[ colNamesForFields.length ];
- for(int i=0; i<colNamesForFields.length; ++i) {
- result[i] = -1;
- }
- int i=-1, fieldLabelCount=0;
- for( String col : colNamesForFields ) {
- ++i;
- if(col == null) {
- continue;
- }
- if( col.trim().isEmpty() ) {
- continue;
- }
- ++fieldLabelCount;
- int loc = tableColNames.indexOf(col);
- if(loc == -1) {
- throw new InvalidColumn("Column '" + col + "' not found in table for input field " + i+1);
- }
- result[i] = loc;
- }
- if(fieldLabelCount>tableColNames.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<fieldToColMapping.length; ++i) {
- int newIndex = fieldToColMapping[i];
- if(newIndex != -1) {
- reorderedFields[newIndex] = fields[i];
- }
- }
- return join(reorderedFields, getSerdeSeparator());
- }
-
- // handles nulls in items[]
- // TODO: perhaps can be made more efficient by creating a byte[] directly
- private static byte[] join(String[] items, char separator) {
- StringBuilder buff = new StringBuilder(100);
- if(items.length == 0)
- return "".getBytes();
- int i=0;
- for(; i<items.length-1; ++i) {
- if(items[i]!=null) {
- buff.append(items[i]);
- }
- buff.append(separator);
- }
- if(items[i]!=null) {
- buff.append(items[i]);
- }
- return buff.toString().getBytes();
- }
-
- protected ArrayList<String> 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<String> getCols(Table table) {
- List<FieldSchema> cols = table.getSd().getCols();
- ArrayList<String> colNames = new ArrayList<String>(cols.size());
- for (FieldSchema col : cols) {
- colNames.add(col.getName().toLowerCase());
- }
- return colNames;
- }
-
- public char getSerdeSeparator() {
- return serdeSeparator;
- }
-}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/HeartBeatFailure.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/HeartBeatFailure.java b/streaming/src/java/org/apache/hive/streaming/HeartBeatFailure.java
deleted file mode 100644
index b1f9520..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-import java.util.Collection;
-import java.util.Set;
-
-public class HeartBeatFailure extends StreamingException {
- private Collection<Long> abortedTxns;
- private Collection<Long> nosuchTxns;
-
- public HeartBeatFailure(Collection<Long> abortedTxns, Set<Long> nosuchTxns) {
- super("Heart beat error. InvalidTxns: " + nosuchTxns + ". AbortedTxns: " + abortedTxns);
- this.abortedTxns = abortedTxns;
- this.nosuchTxns = nosuchTxns;
- }
-}
[7/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
HIVE-19211: New streaming ingest API and support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene Koifman)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/bf8d305a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bf8d305a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bf8d305a
Branch: refs/heads/master
Commit: bf8d305a6fd435b3b510dc3f78b5f3329acbe423
Parents: 46c5580
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed May 2 09:41:43 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed May 2 09:41:43 2018 -0700
----------------------------------------------------------------------
.../org/apache/hadoop/hive/conf/HiveConf.java | 10 +
.../streaming/AbstractRecordWriter.java | 5 +-
.../streaming/DelimitedInputWriter.java | 2 +
.../hive/hcatalog/streaming/HiveEndPoint.java | 2 +
.../hive/hcatalog/streaming/RecordWriter.java | 4 +
.../hcatalog/streaming/StreamingConnection.java | 2 +
.../hcatalog/streaming/StrictJsonWriter.java | 2 +
.../hcatalog/streaming/StrictRegexWriter.java | 2 +
.../hcatalog/streaming/TransactionBatch.java | 3 +-
.../streaming/mutate/HiveConfFactory.java | 5 +-
.../mutate/UgiMetaStoreClientFactory.java | 2 +
.../streaming/mutate/client/AcidTable.java | 2 +
.../mutate/client/AcidTableSerializer.java | 2 +
.../streaming/mutate/client/MutatorClient.java | 2 +
.../mutate/client/MutatorClientBuilder.java | 5 +-
.../streaming/mutate/client/Transaction.java | 4 +
.../streaming/mutate/client/lock/Lock.java | 2 +
.../mutate/client/lock/LockFailureListener.java | 5 +-
.../hive/hcatalog/streaming/mutate/package.html | 2 +-
.../mutate/worker/BucketIdResolver.java | 5 +-
.../mutate/worker/BucketIdResolverImpl.java | 2 +
.../streaming/mutate/worker/Mutator.java | 2 +
.../mutate/worker/MutatorCoordinator.java | 2 +
.../worker/MutatorCoordinatorBuilder.java | 5 +-
.../streaming/mutate/worker/MutatorFactory.java | 4 +
.../streaming/mutate/worker/MutatorImpl.java | 5 +-
.../mutate/worker/PartitionHelper.java | 5 +-
.../mutate/worker/RecordInspector.java | 5 +-
.../mutate/worker/RecordInspectorImpl.java | 2 +
.../hive/hcatalog/streaming/package-info.java | 19 -
.../hive/hcatalog/streaming/TestStreaming.java | 121 +-
itests/hive-unit/pom.xml | 5 +
.../hive/ql/txn/compactor/TestCompactor.java | 761 ++++---
.../hadoop/hive/metastore/HiveClientCache.java | 536 +++++
.../hive/metastore/HiveMetaStoreUtils.java | 35 +
.../hadoop/hive/ql/io/orc/OrcRecordUpdater.java | 24 +-
.../hadoop/hive/ql/lockmgr/DbTxnManager.java | 4 +-
.../apache/hadoop/hive/ql/metadata/Hive.java | 6 +-
.../apache/hadoop/hive/serde2/JsonSerDe.java | 669 ++++++
.../hive/metastore/utils/MetaStoreUtils.java | 8 +-
streaming/pom.xml | 42 +-
.../hive/streaming/AbstractRecordWriter.java | 460 ++--
.../apache/hive/streaming/ConnectionError.java | 6 +-
.../apache/hive/streaming/ConnectionInfo.java | 76 +
.../hive/streaming/DelimitedInputWriter.java | 331 ---
.../apache/hive/streaming/HeartBeatFailure.java | 33 -
.../org/apache/hive/streaming/HiveEndPoint.java | 1117 ---------
.../hive/streaming/HiveStreamingConnection.java | 1039 +++++++++
.../hive/streaming/ImpersonationFailed.java | 25 -
.../apache/hive/streaming/InvalidColumn.java | 26 -
.../apache/hive/streaming/InvalidPartition.java | 28 -
.../org/apache/hive/streaming/InvalidTable.java | 8 +-
.../hive/streaming/InvalidTransactionState.java | 25 +
.../hive/streaming/InvalidTrasactionState.java | 26 -
.../hive/streaming/PartitionCreationFailed.java | 4 +-
.../apache/hive/streaming/PartitionHandler.java | 33 +
.../apache/hive/streaming/PartitionInfo.java | 58 +
.../hive/streaming/QueryFailedException.java | 28 -
.../org/apache/hive/streaming/RecordWriter.java | 41 +-
.../hive/streaming/SerializationError.java | 2 +-
.../hive/streaming/StreamingConnection.java | 59 +-
.../hive/streaming/StreamingException.java | 2 +-
.../hive/streaming/StreamingIOFailure.java | 4 +-
.../streaming/StrictDelimitedInputWriter.java | 110 +
.../apache/hive/streaming/StrictJsonWriter.java | 117 +-
.../hive/streaming/StrictRegexWriter.java | 130 +-
.../apache/hive/streaming/TransactionBatch.java | 125 --
.../streaming/TransactionBatchUnAvailable.java | 25 -
.../apache/hive/streaming/TransactionError.java | 4 +-
.../streaming/TestDelimitedInputWriter.java | 73 -
.../apache/hive/streaming/TestStreaming.java | 2115 +++++++++++-------
.../TestStreamingDynamicPartitioning.java | 921 ++++++++
72 files changed, 5799 insertions(+), 3582 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index fe7b23f..6358ff3 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -665,6 +665,16 @@ public class HiveConf extends Configuration {
@Deprecated
METASTORE_CAPABILITY_CHECK("hive.metastore.client.capability.check", true,
"Whether to check client capabilities for potentially breaking API usage."),
+ METASTORE_CLIENT_CACHE_ENABLED("hive.metastore.client.cache.enabled", false,
+ "Whether to enable metastore client cache"),
+ METASTORE_CLIENT_CACHE_EXPIRY_TIME("hive.metastore.client.cache.expiry.time", "120s",
+ new TimeValidator(TimeUnit.SECONDS), "Expiry time for metastore client cache"),
+ METASTORE_CLIENT_CACHE_INITIAL_CAPACITY("hive.metastore.client.cache.initial.capacity", 50,
+ "Initial capacity for metastore client cache"),
+ METASTORE_CLIENT_CACHE_MAX_CAPACITY("hive.metastore.client.cache.max.capacity", 50,
+ "Max capacity for metastore client cache"),
+ METASTORE_CLIENT_CACHE_STATS_ENABLED("hive.metastore.client.cache.stats.enabled", false,
+ "Whether to enable metastore client cache stats"),
METASTORE_FASTPATH("hive.metastore.fastpath", false,
"Used to avoid all of the proxies and object copies in the metastore. Note, if this is " +
"set, you MUST use a local metastore (hive.metastore.uris must be empty) otherwise " +
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
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
index 2a68220..5e12254 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/AbstractRecordWriter.java
@@ -53,7 +53,10 @@ import java.util.HashSet;
import java.util.List;
import java.util.Properties;
-
+/**
+ * @deprecated as of Hive 3.0.0, replaced by {@link org.apache.hive.streaming.AbstractRecordWriter}
+ */
+@Deprecated
public abstract class AbstractRecordWriter implements RecordWriter {
static final private Logger LOG = LoggerFactory.getLogger(AbstractRecordWriter.class.getName());
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
----------------------------------------------------------------------
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
index 999c37e..32dae45 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/DelimitedInputWriter.java
@@ -46,7 +46,9 @@ 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 {@link org.apache.hive.streaming.StrictDelimitedInputWriter}
*/
+@Deprecated
public class DelimitedInputWriter extends AbstractRecordWriter {
private final boolean reorderingNeeded;
private String delimiter;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
----------------------------------------------------------------------
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
index 8582e9a..3ee19dd 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
@@ -60,7 +60,9 @@ import java.util.List;
* 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 {@link org.apache.hive.streaming.HiveStreamingConnection}
*/
+@Deprecated
public class HiveEndPoint {
public final String metaStoreUri;
public final String database;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java
----------------------------------------------------------------------
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
index a9bcd9f..19078d2 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/RecordWriter.java
@@ -19,6 +19,10 @@
package org.apache.hive.hcatalog.streaming;
+/**
+ * @deprecated as of Hive 3.0.0, replaced by {@link org.apache.hive.streaming.RecordWriter}
+ */
+@Deprecated
public interface RecordWriter {
/** Writes using a hive RecordUpdater
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
----------------------------------------------------------------------
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
index c2b5157..0de8abc 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StreamingConnection.java
@@ -24,7 +24,9 @@ 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 {@link org.apache.hive.streaming.HiveStreamingConnection}
*/
+@Deprecated
public interface StreamingConnection {
/**
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
----------------------------------------------------------------------
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
index 4d92c55..48e7e49 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictJsonWriter.java
@@ -37,7 +37,9 @@ 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 {@link org.apache.hive.streaming.StrictJsonWriter}
*/
+@Deprecated
public class StrictJsonWriter extends AbstractRecordWriter {
private JsonSerDe serde;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java
----------------------------------------------------------------------
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
index ae25662..f0540e0 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/StrictRegexWriter.java
@@ -41,7 +41,9 @@ 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 {@link org.apache.hive.streaming.StrictRegexWriter}
*/
+@Deprecated
public class StrictRegexWriter extends AbstractRecordWriter {
private RegexSerDe serde;
private final StructObjectInspector recordObjInspector;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
----------------------------------------------------------------------
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
index 1208377..400fd49 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/TransactionBatch.java
@@ -30,8 +30,9 @@ import java.util.Collection;
* 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 {@link org.apache.hive.streaming.HiveStreamingConnection}
*/
+@Deprecated
public interface TransactionBatch {
enum TxnState {
INACTIVE("I"), OPEN("O"), COMMITTED("C"), ABORTED("A");
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java
----------------------------------------------------------------------
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
index f23a78b..ebe032d 100644
--- 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
@@ -22,7 +22,10 @@ import org.apache.hadoop.hive.conf.HiveConf;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/** Creates/configures {@link HiveConf} instances with required ACID attributes. */
+/** 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java
----------------------------------------------------------------------
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
index 328b6c0..615fc1a 100644
--- 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
@@ -40,7 +40,9 @@ 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<Method> I_META_STORE_CLIENT_METHODS = getIMetaStoreClientMethods();
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java
----------------------------------------------------------------------
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
index 50ba0c7..40de497 100644
--- 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
@@ -25,7 +25,9 @@ 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;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java
----------------------------------------------------------------------
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
index 98f9f40..43ac527 100644
--- 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
@@ -37,7 +37,9 @@ 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java
----------------------------------------------------------------------
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
index 8ba6cf6..11664f6 100644
--- 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
@@ -39,7 +39,9 @@ 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java
----------------------------------------------------------------------
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
index 30a060c..1575d8d 100644
--- 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
@@ -32,7 +32,10 @@ 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. */
+/** Convenience class for building {@link MutatorClient} instances.
+ * @deprecated as of Hive 3.0.0
+ */
+@Deprecated
public class MutatorClientBuilder {
private final Map<String, AcidTable> tables = new HashMap<>();
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java
----------------------------------------------------------------------
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
index fa03f74..e1c6735 100644
--- 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
@@ -27,6 +27,10 @@ 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
----------------------------------------------------------------------
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
index abbb125..52eb613 100644
--- 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
@@ -43,7 +43,9 @@ 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java
----------------------------------------------------------------------
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
index 55502b4..a3845ea 100644
--- 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
@@ -20,7 +20,10 @@ 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. */
+/** 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
----------------------------------------------------------------------
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
index d133c46..7bc75c0 100644
--- 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
@@ -13,7 +13,7 @@
<body>
<h1>HCatalog Streaming Mutation API -- high level description</h1>
-
+<h2>@deprecated as of Hive 3.0.0</h2>
<h2>Background</h2>
<p>
In certain data processing use cases it is necessary to modify existing
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java
----------------------------------------------------------------------
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
index 5dd0b8e..3432baa 100644
--- 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
@@ -17,7 +17,10 @@
*/
package org.apache.hive.hcatalog.streaming.mutate.worker;
-/** Computes and appends bucket ids to records that are due to be inserted. */
+/** 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java
----------------------------------------------------------------------
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
index 7c2cade..1d51d85 100644
--- 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
@@ -30,7 +30,9 @@ 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;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java
----------------------------------------------------------------------
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
index 7126a88..e6f968e 100644
--- 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
@@ -24,7 +24,9 @@ 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;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
----------------------------------------------------------------------
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
index ad14c72..67785d0 100644
--- 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
@@ -52,7 +52,9 @@ import org.slf4j.LoggerFactory;
* 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);
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
----------------------------------------------------------------------
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
index 80b90a2..698ba7c 100644
--- 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
@@ -32,7 +32,10 @@ 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. */
+/** Convenience class for building {@link MutatorCoordinator} instances.
+ * @deprecated as of Hive 3.0.0
+ */
+@Deprecated
public class MutatorCoordinatorBuilder {
private HiveConf configuration;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java
----------------------------------------------------------------------
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
index da7558f..d3d3210 100644
--- 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
@@ -22,6 +22,10 @@ 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)
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java
----------------------------------------------------------------------
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
index 84c477f..1e0cb72 100644
--- 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
@@ -28,7 +28,10 @@ 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. */
+/** 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;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java
----------------------------------------------------------------------
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
index 165bc5e..d064b0c 100644
--- 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
@@ -22,7 +22,10 @@ import java.util.List;
import org.apache.hadoop.fs.Path;
-/** Implementations are responsible for creating and obtaining path information about partitions. */
+/** 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. */
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java
----------------------------------------------------------------------
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
index 55add67..5d1f175 100644
--- 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
@@ -19,7 +19,10 @@ 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. */
+/** 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. */
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java
----------------------------------------------------------------------
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
index 9438e13..37329c3 100644
--- 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
@@ -29,7 +29,9 @@ 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;
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package-info.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package-info.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package-info.java
deleted file mode 100644
index 36d6b13..0000000
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/package-info.java
+++ /dev/null
@@ -1,19 +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.
- */
-@Deprecated // use org.apache.hive.streaming instead
-package org.apache.hive.hcatalog.streaming;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
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
index 90dbdac..13aa5e9 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -439,68 +439,75 @@ public class TestStreaming {
String tableLoc = "'" + dbUri + Path.SEPARATOR + "streamedtable" + "'";
String tableLoc2 = "'" + dbUri + Path.SEPARATOR + "finaltable" + "'";
String tableLoc3 = "'" + dbUri + Path.SEPARATOR + "nobucket" + "'";
+ // disabling vectorization as this test yields incorrect results with vectorization
+ conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false);
+ 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();
- 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());
- }
+ for (String record : records) {
+ txnBatch.write(record.toString().getBytes());
+ }
- txnBatch.commit();
- txnBatch.close();
- connection.close();
+ txnBatch.commit();
+ txnBatch.close();
+ connection.close();
- ArrayList<String> res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2");
- for (String re : res1) {
- System.out.println(re);
- }
+ ArrayList<String> 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<String> res2 = queryTable(driver, "select row__id.bucketid,* from finaltable where row__id.bucketid<>bucketid");
- for (String s : res2) {
- LOG.error(s);
+ driver.run("insert into nobucket select row__id.bucketid,* from streamedtable");
+ runDDL(driver, " insert into finaltable select * from nobucket");
+ ArrayList<String> 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);
}
- Assert.assertTrue(res2.isEmpty());
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/itests/hive-unit/pom.xml
----------------------------------------------------------------------
diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml
index 3ae7f2f..b51ebf2 100644
--- a/itests/hive-unit/pom.xml
+++ b/itests/hive-unit/pom.xml
@@ -76,6 +76,11 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.hive.hcatalog</groupId>
+ <artifactId>hive-hcatalog-streaming</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-streaming</artifactId>
<version>${project.version}</version>
[2/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
index 0ec3048..a6fdd66 100644
--- a/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
+++ b/streaming/src/test/org/apache/hive/streaming/TestStreaming.java
@@ -35,6 +35,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -53,12 +54,10 @@ 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.MetaException;
-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;
@@ -90,13 +89,11 @@ 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;
@@ -115,6 +112,7 @@ public class TestStreaming {
public static class RawFileSystem extends RawLocalFileSystem {
private static final URI NAME;
+
static {
try {
NAME = new URI("raw:///");
@@ -128,12 +126,16 @@ public class TestStreaming {
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);
+ throw new FileNotFoundException("Can'table find " + path);
}
// get close enough
short mod = 0;
@@ -147,32 +149,30 @@ public class TestStreaming {
mod |= 0111;
}
return new FileStatus(file.length(), file.isDirectory(), 1, 1024,
- file.lastModified(), file.lastModified(),
- FsPermission.createImmutable(mod), "owen", "users", path);
+ 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 static HiveConf conf = null;
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<String> partitionVals;
+ private final static String[] fieldNames = new String[]{COL1, COL2};
+ static List<String> 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};
+ private final static String[] fieldNames2 = new String[]{COL1, COL2};
// for bucket join testing
@@ -201,13 +201,9 @@ public class TestStreaming {
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");
+ 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();
@@ -229,12 +225,13 @@ public class TestStreaming {
// 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[] 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);
+ partLoc = createDbAndTable(driver, dbName, tblName, partitionVals, colNames, colTypes, bucketCols, partNames, loc1,
+ 1);
dropDB(msClient, dbName2);
String loc2 = dbFolder.newFolder(dbName2 + ".db").toString();
@@ -249,19 +246,11 @@ public class TestStreaming {
}
@After
- public void cleanup() throws Exception {
+ public void cleanup() {
msClient.close();
driver.close();
}
- private static List<FieldSchema> getPartitionKeys() {
- List<FieldSchema> fields = new ArrayList<FieldSchema>();
- // 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";
@@ -301,43 +290,48 @@ public class TestStreaming {
")\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')");
+ "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<String> partitionVals = new ArrayList<String>();
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();
-
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testing5")
+ .withTable("store_sales")
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
StringBuilder row = new StringBuilder();
- for(int i = 0; i < 10; i++) {
- for(int ints = 0; ints < 11; ints++) {
+ 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++) {
+ for (int decs = 0; decs < 12; decs++) {
row.append(i + 0.1).append(',');
}
row.setLength(row.length() - 1);
- txnBatch.write(row.toString().getBytes());
+ connection.write(row.toString().getBytes());
}
- txnBatch.commit();
- txnBatch.close();
+ connection.commitTransaction();
connection.close();
ArrayList<String> res = queryTable(driver, "select row__id.bucketid, * from testing5.store_sales");
@@ -352,35 +346,41 @@ public class TestStreaming {
@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, "create table default.streamingnobuckets (a string, b string) stored as orc " +
+ "TBLPROPERTIES('transactional'='true')");
queryTable(driver, "insert into default.streamingnobuckets values('foo','bar')");
List<String> 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());
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("Default")
+ .withTable("streamingNoBuckets")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withTransactionBatchSize(2)
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.write("a1,b2".getBytes());
+ connection.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();
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("a5,b6".getBytes());
+ connection.write("a7,b8".getBytes());
+ connection.commitTransaction();
+ connection.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");
+ 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"));
@@ -404,7 +404,7 @@ public class TestStreaming {
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");
+ 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/bucket_00000"));
@@ -416,6 +416,152 @@ public class TestStreaming {
Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000005/bucket_00000"));
}
+ @Test
+ public void testAllTypesDelimitedWriter() throws Exception {
+ queryTable(driver, "drop table if exists default.alltypes");
+ queryTable(driver,
+ "create table if not exists default.alltypes ( bo boolean, ti tinyint, si smallint, i int, bi bigint, " +
+ "f float, d double, de decimal(10,3), ts timestamp, da date, s string, c char(5), vc varchar(5), " +
+ "m map<string, string>, l array<int>, st struct<c1:int, c2:string> ) " +
+ "stored as orc TBLPROPERTIES('transactional'='true')");
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter('|')
+ .withCollectionDelimiter(',')
+ .withMapKeyDelimiter(':')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("default")
+ .withTable("alltypes")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withTransactionBatchSize(2)
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+
+ String row1 = "true|10|100|1000|10000|4.0|20.0|4.2222|1969-12-31 " +
+ "15:59:58.174|1970-01-01|string|hello|hello|k1:v1|100,200|10,foo";
+ String row2 = "false|20|200|2000|20000|8.0|40.0|2.2222|1970-12-31 15:59:58.174|1971-01-01|abcd|world|world|" +
+ "k4:v4|200,300|20,bar";
+ connection.beginTransaction();
+ connection.write(row1.getBytes());
+ connection.write(row2.getBytes());
+ connection.commitTransaction();
+ connection.close();
+
+ List<String> rs = queryTable(driver, "select ROW__ID, bo, ti, si, i, bi, f, d, de, ts, da, s, c, vc, m, l, st," +
+ " INPUT__FILE__NAME from default.alltypes order by ROW__ID");
+ Assert.assertEquals(2, rs.size());
+ String gotRow1 = rs.get(0);
+ String expectedPrefixRow1 = "{\"writeid\":1,\"bucketid\":536870912," +
+ "\"rowid\":0}\ttrue\t10\t100\t1000\t10000\t4.0\t20.0\t4.222\t1969-12-31 15:59:58.174\t1970-01-01\tstring" +
+ "\thello\thello\t{\"k1\":\"v1\"}\t[100,200]\t{\"c1\":10,\"c2\":\"foo\"}";
+ String expectedSuffixRow1 = "alltypes/delta_0000001_0000002/bucket_00000";
+ String gotRow2 = rs.get(1);
+ String expectedPrefixRow2 = "{\"writeid\":1,\"bucketid\":536870912," +
+ "\"rowid\":1}\tfalse\t20\t200\t2000\t20000\t8.0\t40.0\t2.222\t1970-12-31 15:59:58.174\t1971-01-01\tabcd" +
+ "\tworld\tworld\t{\"k4\":\"v4\"}\t[200,300]\t{\"c1\":20,\"c2\":\"bar\"}";
+ String expectedSuffixRow2 = "alltypes/delta_0000001_0000002/bucket_00000";
+ Assert.assertTrue(gotRow1, gotRow1.startsWith(expectedPrefixRow1));
+ Assert.assertTrue(gotRow1, gotRow1.endsWith(expectedSuffixRow1));
+ Assert.assertTrue(gotRow2, gotRow2.startsWith(expectedPrefixRow2));
+ Assert.assertTrue(gotRow2, gotRow2.endsWith(expectedSuffixRow2));
+ }
+
+ @Test
+ public void testAutoRollTransactionBatch() throws Exception {
+ queryTable(driver, "drop table if exists default.streamingnobuckets");
+ queryTable(driver, "create table default.streamingnobuckets (a string, b string) stored as orc " +
+ "TBLPROPERTIES('transactional'='true')");
+ queryTable(driver, "insert into default.streamingnobuckets values('foo','bar')");
+ List<String> rs = queryTable(driver, "select * from default.streamingnobuckets");
+ Assert.assertEquals(1, rs.size());
+ Assert.assertEquals("foo\tbar", rs.get(0));
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("default")
+ .withTable("streamingnobuckets")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .withTransactionBatchSize(2)
+ .connect();
+
+ connection.beginTransaction();
+ connection.write("a1,b2".getBytes());
+ connection.write("a3,b4".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("a5,b6".getBytes());
+ connection.write("a7,b8".getBytes());
+ connection.commitTransaction();
+ // should have rolled over to next transaction batch
+ connection.beginTransaction();
+ connection.write("a9,b10".getBytes());
+ connection.write("a11,b12".getBytes());
+ connection.commitTransaction();
+ connection.beginTransaction();
+ connection.write("a13,b14".getBytes());
+ connection.write("a15,b16".getBytes());
+ connection.commitTransaction();
+ connection.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"));
+ 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"));
+
+ Assert.assertTrue(rs.get(5), rs.get(5).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":0}\ta9\tb10"));
+ Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/delta_0000004_0000005/bucket_00000"));
+ Assert.assertTrue(rs.get(6), rs.get(6).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":1}\ta11\tb12"));
+ Assert.assertTrue(rs.get(6), rs.get(6).endsWith("streamingnobuckets/delta_0000004_0000005/bucket_00000"));
+ Assert.assertTrue(rs.get(7), rs.get(7).startsWith("{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\ta13\tb14"));
+ Assert.assertTrue(rs.get(7), rs.get(7).endsWith("streamingnobuckets/delta_0000004_0000005/bucket_00000"));
+ Assert.assertTrue(rs.get(8), rs.get(8).startsWith("{\"writeid\":5,\"bucketid\":536870912,\"rowid\":1}\ta15\tb16"));
+ Assert.assertTrue(rs.get(8), rs.get(8).endsWith("streamingnobuckets/delta_0000004_0000005/bucket_00000"));
+
+ queryTable(driver, "update default.streamingnobuckets set a=0, b=0 where a='a7'");
+ queryTable(driver, "delete from default.streamingnobuckets where a='a1'");
+ queryTable(driver, "update default.streamingnobuckets set a=0, b=0 where a='a15'");
+ queryTable(driver, "delete from default.streamingnobuckets where a='a9'");
+ 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, "0\t0", rs.get(row++));
+ Assert.assertEquals("at row=" + row, "a11\tb12", rs.get(row++));
+ Assert.assertEquals("at row=" + row, "a13\tb14", 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_0000009/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_0000009/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_0000009/bucket_00000"));
+ Assert.assertTrue(rs.get(3), rs.get(3).startsWith("{\"writeid\":4,\"bucketid\":536870912,\"rowid\":1}\ta11\tb12"));
+ Assert.assertTrue(rs.get(3), rs.get(3).endsWith("streamingnobuckets/base_0000009/bucket_00000"));
+ Assert.assertTrue(rs.get(4), rs.get(4).startsWith("{\"writeid\":5,\"bucketid\":536870912,\"rowid\":0}\ta13\tb14"));
+ Assert.assertTrue(rs.get(4), rs.get(4).endsWith("streamingnobuckets/base_0000009/bucket_00000"));
+ Assert.assertTrue(rs.get(5), rs.get(5).startsWith("{\"writeid\":6,\"bucketid\":536870912,\"rowid\":0}\t0\t0"));
+ Assert.assertTrue(rs.get(5), rs.get(5).endsWith("streamingnobuckets/base_0000009/bucket_00000"));
+ }
+
/**
* this is a clone from TestTxnStatement2....
*/
@@ -436,71 +582,83 @@ public class TestStreaming {
int bucketCount = 100;
String dbUri = "raw://" + new Path(dbFolder.newFolder().toString()).toUri().toString();
- String tableLoc = "'" + dbUri + Path.SEPARATOR + "streamedtable" + "'";
+ String tableLoc = "'" + dbUri + Path.SEPARATOR + "streamedtable" + "'";
String tableLoc2 = "'" + dbUri + Path.SEPARATOR + "finaltable" + "'";
String tableLoc3 = "'" + dbUri + Path.SEPARATOR + "nobucket" + "'";
- 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());
- }
+ // disabling vectorization as this test yields incorrect results with vectorization
+ conf.setBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, false);
+ 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"
+ };
+
+ StrictDelimitedInputWriter wr = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("streamedtable")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(wr)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+
+ for (String record : records) {
+ connection.write(record.getBytes());
+ }
- txnBatch.commit();
- txnBatch.close();
- connection.close();
+ connection.commitTransaction();
+ connection.close();
- ArrayList<String> res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2");
- for (String re : res1) {
- System.out.println(re);
- }
+ ArrayList<String> res1 = queryTable(driver, "select row__id.bucketid, * from streamedtable order by key2");
+ for (String re : res1) {
+ LOG.error(re);
+ }
- driver.run("insert into nobucket select row__id.bucketid,* from streamedtable");
- runDDL(driver, " insert into finaltable select * from nobucket");
- ArrayList<String> res2 = queryTable(driver, "select row__id.bucketid,* from finaltable where row__id.bucketid<>bucketid");
- for (String s : res2) {
- LOG.error(s);
+ driver.run("insert into nobucket select row__id.bucketid,* from streamedtable");
+ runDDL(driver, "insert into finaltable select * from nobucket");
+ ArrayList<String> 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);
}
- Assert.assertTrue(res2.isEmpty());
}
@@ -512,32 +670,53 @@ public class TestStreaming {
String tbl1 = "validation1";
String tbl2 = "validation2";
- String tableLoc = "'" + dbUri + Path.SEPARATOR + tbl1 + "'";
+ 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')") ;
+ + 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')") ;
-
+ + bucketCount + " buckets stored as orc location " + tableLoc2 + " TBLPROPERTIES ('transactional'='false')");
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = null;
try {
- HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "validation1", null);
- endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("validation2")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
Assert.assertTrue("InvalidTable exception was not thrown", false);
} catch (InvalidTable e) {
// expecting this exception
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
}
try {
- HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, "testBucketing3", "validation2", null);
- endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase("testBucketing3")
+ .withTable("validation2")
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
Assert.assertTrue("InvalidTable exception was not thrown", false);
} catch (InvalidTable e) {
// expecting this exception
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
}
}
@@ -547,7 +726,7 @@ public class TestStreaming {
*/
@Deprecated
private void checkDataWritten(Path partitionPath, long minTxn, long maxTxn, int buckets, int numExpectedFiles,
- String... records) throws Exception {
+ String... records) throws Exception {
ValidWriteIdList writeIds = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName));
AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, writeIds);
Assert.assertEquals(0, dir.getObsolete().size());
@@ -585,7 +764,7 @@ public class TestStreaming {
InputSplit[] splits = inf.getSplits(job, buckets);
Assert.assertEquals(numExpectedFiles, splits.length);
org.apache.hadoop.mapred.RecordReader<NullWritable, OrcStruct> rr =
- inf.getRecordReader(splits[0], job, Reporter.NULL);
+ inf.getRecordReader(splits[0], job, Reporter.NULL);
NullWritable key = rr.createKey();
OrcStruct value = rr.createValue();
@@ -595,12 +774,13 @@ public class TestStreaming {
}
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
+ * @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 {
+ String validationQuery, boolean vectorize, String... records) throws Exception {
ValidWriteIdList txns = msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName));
AcidUtils.Directory dir = AcidUtils.getAcidState(partitionPath, conf, txns);
Assert.assertEquals(0, dir.getObsolete().size());
@@ -626,12 +806,13 @@ public class TestStreaming {
Assert.assertEquals(minTxn, min);
Assert.assertEquals(maxTxn, max);
boolean isVectorizationEnabled = conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED);
- if(vectorize) {
+ 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()) {
+ 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<String> actualResult = queryTable(driver, validationQuery);
@@ -656,35 +837,44 @@ public class TestStreaming {
@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
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
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));
- }
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.close();
// For unpartitioned table, partition values are specified
try {
- endPt = new HiveEndPoint(metaStoreURI, dbName2, tblName2, partitionVals);
- connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
Assert.assertTrue("ConnectionError was not thrown", false);
connection.close();
} catch (ConnectionError e) {
// expecting this exception
- String errMsg = "specifies partitions for unpartitioned table";
+ String errMsg = "specifies partitions for un-partitioned table";
Assert.assertTrue(e.toString().endsWith(errMsg));
}
}
@@ -695,417 +885,474 @@ public class TestStreaming {
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
- }
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(newPartVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
// Create partition
- Assert.assertNotNull(endPt.newConnection(true, "UT_" + Thread.currentThread().getName()));
+ Assert.assertNotNull(connection);
// Ensure partition is present
- Partition p = msClient.getPartition(endPt.database, endPt.table, endPt.partitionVals);
+ Partition p = msClient.getPartition(dbName, tblName, 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();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+ connection.commitTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
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();
+ writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.commitTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
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();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
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);
+ conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 2, TimeUnit.MILLISECONDS);
AcidHouseKeeperService houseKeeperService = new AcidHouseKeeperService();
houseKeeperService.setConf(conf);
houseKeeperService.run();
try {
//should fail because the TransactionBatch timed out
- txnBatch.commit();
- }
- catch(TransactionError e) {
+ connection.commitTransaction();
+ } 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();
+ connection.close();
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
+ connection.commitTransaction();
+ connection.beginTransaction();
houseKeeperService.run();
try {
//should fail because the TransactionBatch timed out
- txnBatch.commit();
- }
- catch(TransactionError e) {
+ connection.commitTransaction();
+ } 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();
+ int transactionBatch = 20;
+ conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 200, TimeUnit.MILLISECONDS);
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withTransactionBatchSize(transactionBatch)
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ try {
+ connection.beginTransaction();
+ ShowLocksRequest request = new ShowLocksRequest();
+ request.setDbname(dbName2);
+ request.setTablename(tblName2);
+ ShowLocksResponse response = msClient.showLocks(request);
+ Assert.assertEquals("Wrong number of locks: " + response, 1, response.getLocks().size());
+ ShowLocksResponseElement lock = response.getLocks().get(0);
+ long acquiredAt = lock.getAcquiredat();
+ long heartbeatAt = lock.getLastheartbeat();
+ 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'table match", acquiredAt, lock.getAcquiredat());
+ Assert.assertTrue("Expected new heartbeat (" + lock.getLastheartbeat() +
+ ") == old heartbeat(" + heartbeatAt + ")", lock.getLastheartbeat() == heartbeatAt);
+ for (int i = 0; i < transactionBatch * 3; i++) {
+ connection.beginTransaction();
+ if (i % 10 == 0) {
+ connection.abortTransaction();
+ } else {
+ connection.commitTransaction();
+ }
+ Thread.sleep(10);
}
+ } finally {
+ conf.unset(HiveConf.ConfVars.HIVE_TXN_TIMEOUT.varname);
+ connection.close();
}
-
}
+
@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();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.abortTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.ABORTED
+ , connection.getCurrentTransactionState());
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();
+ writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.abortTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.ABORTED
+ , connection.getCurrentTransactionState());
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);
+ public void testTransactionBatchCommitDelimited() throws Exception {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(10)
+ .connect();
// 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();
+ connection.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("1,Hello streaming".getBytes());
+ connection.commitTransaction();
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
- Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
- , txnBatch.getCurrentTransactionState());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
// 2nd Txn
- txnBatch.beginNextTransaction();
- Assert.assertEquals(TransactionBatch.TxnState.OPEN
- , txnBatch.getCurrentTransactionState());
- txnBatch.write("2,Welcome to streaming".getBytes());
+ connection.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("2,Welcome to streaming".getBytes());
// data should not be visible
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
- txnBatch.commit();
+ connection.commitTransaction();
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
"{2, Welcome to streaming}");
- txnBatch.close();
- Assert.assertEquals(TransactionBatch.TxnState.INACTIVE
- , txnBatch.getCurrentTransactionState());
-
-
connection.close();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.INACTIVE
+ , connection.getCurrentTransactionState());
- // 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);
+ // To Unpartitioned table
+ writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .connect();
// 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.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("1,Hello streaming".getBytes());
+ connection.commitTransaction();
+
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.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());
+ public void testTransactionBatchCommitRegex() throws Exception {
String regex = "([^,]*),(.*)";
- StrictRegexWriter writer = new StrictRegexWriter(regex, endPt, conf, connection);
+ StrictRegexWriter writer = StrictRegexWriter.newBuilder()
+ .withRegex(regex)
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(10)
+ .connect();
// 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();
+ connection.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("1,Hello streaming".getBytes());
+ connection.commitTransaction();
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
- Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
- , txnBatch.getCurrentTransactionState());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
// 2nd Txn
- txnBatch.beginNextTransaction();
- Assert.assertEquals(TransactionBatch.TxnState.OPEN
- , txnBatch.getCurrentTransactionState());
- txnBatch.write("2,Welcome to streaming".getBytes());
+ connection.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("2,Welcome to streaming".getBytes());
// data should not be visible
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}");
- txnBatch.commit();
+ connection.commitTransaction();
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
"{2, Welcome to streaming}");
- txnBatch.close();
- Assert.assertEquals(TransactionBatch.TxnState.INACTIVE
- , txnBatch.getCurrentTransactionState());
-
-
connection.close();
-
+ Assert.assertEquals(HiveStreamingConnection.TxnState.INACTIVE
+ , connection.getCurrentTransactionState());
// 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);
+ writer = StrictRegexWriter.newBuilder()
+ .withRegex(regex)
+ .build();
+
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName2)
+ .withTable(tblName2)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .connect();
// 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.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write("1:Hello streaming".getBytes());
+ connection.commitTransaction();
+
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.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);
+ public void testTransactionBatchCommitJson() throws Exception {
+ StrictJsonWriter writer = StrictJsonWriter.newBuilder()
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .withTransactionBatchSize(10)
+ .connect();
// 1st Txn
- TransactionBatch txnBatch = connection.fetchTransactionBatch(10, writer);
- txnBatch.beginNextTransaction();
- Assert.assertEquals(TransactionBatch.TxnState.OPEN
- , txnBatch.getCurrentTransactionState());
+ connection.beginTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
String rec1 = "{\"id\" : 1, \"msg\": \"Hello streaming\"}";
- txnBatch.write(rec1.getBytes());
- txnBatch.commit();
+ connection.write(rec1.getBytes());
+ connection.commitTransaction();
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());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
connection.close();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.INACTIVE
+ , connection.getCurrentTransactionState());
+
List<String> 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());
-
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ connection.beginTransaction();
// 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());
+ int batch = 0;
+ int initialCount = connection.remainingTransactions();
+ while (connection.remainingTransactions() > 0) {
+ connection.beginTransaction();
+ Assert.assertEquals(--initialCount, connection.remainingTransactions());
+ for (int rec = 0; rec < 2; ++rec) {
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write((batch * rec + ",Hello streaming").getBytes());
}
- txnBatch.commit();
- Assert.assertEquals(TransactionBatch.TxnState.COMMITTED
- , txnBatch.getCurrentTransactionState());
+ connection.commitTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
++batch;
}
- Assert.assertEquals(0, txnBatch.remainingTransactions());
- txnBatch.close();
-
- Assert.assertEquals(TransactionBatch.TxnState.INACTIVE
- , txnBatch.getCurrentTransactionState());
+ Assert.assertEquals(0, connection.remainingTransactions());
+ connection.close();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.INACTIVE
+ , connection.getCurrentTransactionState());
+
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
// 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());
+ connection.beginTransaction();
+ batch = 0;
+ initialCount = connection.remainingTransactions();
+ while (connection.remainingTransactions() > 0) {
+ connection.beginTransaction();
+ Assert.assertEquals(--initialCount, connection.remainingTransactions());
+ for (int rec = 0; rec < 2; ++rec) {
+ Assert.assertEquals(HiveStreamingConnection.TxnState.OPEN
+ , connection.getCurrentTransactionState());
+ connection.write((batch * rec + ",Hello streaming").getBytes());
}
- txnBatch.abort();
- Assert.assertEquals(TransactionBatch.TxnState.ABORTED
- , txnBatch.getCurrentTransactionState());
+ connection.abortTransaction();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.ABORTED
+ , connection.getCurrentTransactionState());
++batch;
}
- Assert.assertEquals(0, txnBatch.remainingTransactions());
- txnBatch.close();
-
- Assert.assertEquals(TransactionBatch.TxnState.INACTIVE
- , txnBatch.getCurrentTransactionState());
-
+ Assert.assertEquals(0, connection.remainingTransactions());
connection.close();
+
+ Assert.assertEquals(HiveStreamingConnection.TxnState.INACTIVE
+ , connection.getCurrentTransactionState());
}
@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();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.write("1,Hello streaming".getBytes());
+ connection.write("2,Welcome to streaming".getBytes());
+ connection.abortTransaction();
checkNothingWritten(partLoc);
- Assert.assertEquals(TransactionBatch.TxnState.ABORTED
- , txnBatch.getCurrentTransactionState());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.ABORTED
+ , connection.getCurrentTransactionState());
- txnBatch.close();
connection.close();
checkNothingWritten(partLoc);
@@ -1116,123 +1363,162 @@ public class TestStreaming {
@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());
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo(agentInfo)
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .withTransactionBatchSize(10)
+ .connect();
+
+ connection.beginTransaction();
+ connection.write("1,Hello streaming".getBytes());
+ connection.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();
+ connection.abortTransaction();
checkNothingWritten(partLoc);
- Assert.assertEquals(TransactionBatch.TxnState.ABORTED
- , txnBatch.getCurrentTransactionState());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.ABORTED
+ , connection.getCurrentTransactionState());
- txnBatch.beginNextTransaction();
- txnBatch.write("1,Hello streaming".getBytes());
- txnBatch.write("2,Welcome to streaming".getBytes());
- txnBatch.commit();
+ connection.beginTransaction();
+ connection.write("1,Hello streaming".getBytes());
+ connection.write("2,Welcome to streaming".getBytes());
+ connection.commitTransaction();
checkDataWritten(partLoc, 1, 10, 1, 1, "{1, Hello streaming}",
- "{2, Welcome to 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();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(10)
+ .withHiveConf(conf)
+ .connect();
+
+ connection.beginTransaction();
+ connection.write("1,Hello streaming".getBytes());
+ connection.commitTransaction();
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();
+ connection.beginTransaction();
+ connection.write("2,Welcome to streaming".getBytes());
+ connection.commitTransaction();
- checkDataWritten2(partLoc, 1, 10, 1, validationQuery, true, "1\tHello streaming",
- "2\tWelcome to streaming");
+ checkDataWritten2(partLoc, 1, 10, 1, validationQuery, true, "1\tHello streaming",
+ "2\tWelcome to streaming");
- txnBatch.close();
+ connection.close();
+ connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(10)
+ .withHiveConf(conf)
+ .connect();
// 2nd Txn Batch
- txnBatch = connection.fetchTransactionBatch(10, writer);
- txnBatch.beginNextTransaction();
- txnBatch.write("3,Hello streaming - once again".getBytes());
- txnBatch.commit();
+ connection.beginTransaction();
+ connection.write("3,Hello streaming - once again".getBytes());
+ connection.commitTransaction();
- checkDataWritten2(partLoc, 1, 20, 2, validationQuery, false, "1\tHello streaming",
- "2\tWelcome to streaming", "3\tHello streaming - once again");
+ 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();
+ connection.beginTransaction();
+ connection.write("4,Welcome to streaming - once again".getBytes());
+ connection.commitTransaction();
- 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");
+ 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();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
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());
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .withTransactionBatchSize(10)
+ .connect();
// Acquire 1st Txn Batch
- TransactionBatch txnBatch1 = connection.fetchTransactionBatch(10, writer);
- txnBatch1.beginNextTransaction();
+ connection.beginTransaction();
// Acquire 2nd Txn Batch
- DelimitedInputWriter writer2 = new DelimitedInputWriter(fieldNames, ",", endPt);
- TransactionBatch txnBatch2 = connection.fetchTransactionBatch(10, writer2);
- txnBatch2.beginNextTransaction();
+ StrictDelimitedInputWriter writer2 = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection2 = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withRecordWriter(writer2)
+ .withHiveConf(conf)
+ .withTransactionBatchSize(10)
+ .connect();
+ connection2.beginTransaction();
// Interleaved writes to both batches
- txnBatch1.write("1,Hello streaming".getBytes());
- txnBatch2.write("3,Hello streaming - once again".getBytes());
+ connection.write("1,Hello streaming".getBytes());
+ connection2.write("3,Hello streaming - once again".getBytes());
checkNothingWritten(partLoc);
- txnBatch2.commit();
+ connection2.commitTransaction();
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();
+ connection.commitTransaction();
/*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(partLoc, conf,
- msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)));
- for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) {
- for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) {
+ msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)));
+ 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();
@@ -1244,20 +1530,20 @@ public class TestStreaming {
}
}
checkDataWritten2(partLoc, 1, 20, 2,
- validationQuery, false,"1\tHello streaming", "3\tHello streaming - once again");
+ validationQuery, false, "1\tHello streaming", "3\tHello streaming - once again");
- txnBatch1.beginNextTransaction();
- txnBatch1.write("2,Welcome to streaming".getBytes());
+ connection.beginTransaction();
+ connection.write("2,Welcome to streaming".getBytes());
- txnBatch2.beginNextTransaction();
- txnBatch2.write("4,Welcome to streaming - once again".getBytes());
+ connection2.beginTransaction();
+ connection2.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(partLoc, conf, msClient.getValidWriteIds(AcidUtils.getFullTableName(dbName, tblName)));
- for(AcidUtils.ParsedDelta pd : dir.getCurrentDirectories()) {
- for(FileStatus stat : fs.listStatus(pd.getPath(), AcidUtils.bucketFileFilter)) {
+ 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();
@@ -1269,103 +1555,97 @@ public class TestStreaming {
}
}
checkDataWritten2(partLoc, 1, 20, 2,
- validationQuery, true,"1\tHello streaming", "3\tHello streaming - once again");
+ validationQuery, true, "1\tHello streaming", "3\tHello streaming - once again");
- txnBatch1.commit();
+ connection.commitTransaction();
checkDataWritten2(partLoc, 1, 20, 2,
validationQuery, false, "1\tHello streaming",
- "2\tWelcome to streaming",
- "3\tHello streaming - once again");
+ "2\tWelcome to streaming",
+ "3\tHello streaming - once again");
- txnBatch2.commit();
+ connection2.commitTransaction();
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());
+ "2\tWelcome to streaming",
+ "3\tHello streaming - once again",
+ "4\tWelcome to streaming - once again");
- txnBatch1.close();
- txnBatch2.close();
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection.getCurrentTransactionState());
+ Assert.assertEquals(HiveStreamingConnection.TxnState.COMMITTED
+ , connection2.getCurrentTransactionState());
connection.close();
+ connection2.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 {
+ WriterThd(String data) throws Exception {
super("Writer_" + data);
- writer = new DelimitedInputWriter(fieldNames, ",", ep);
- conn = ep.newConnection(false, "UT_" + Thread.currentThread().getName());
+ RecordWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ HiveStreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withStaticPartitionValues(partitionVals)
+ .withRecordWriter(writer)
+ .withHiveConf(conf)
+ .connect();
+ this.conn = connection;
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);
- }
+ setUncaughtExceptionHandler((thread, throwable) -> {
+ error = throwable;
+ LOG.error(connection.toTransactionString());
+ 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();
+ for (int i = 0; i < 10; i++) {
+ conn.beginTransaction();
+ conn.write(data.getBytes());
+ conn.write(data.getBytes());
+ conn.commitTransaction();
} // while
} catch (Exception e) {
throw new RuntimeException(e);
} finally {
- if (txnBatch != null) {
+ if (conn != null) {
try {
- txnBatch.close();
+ conn.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);
- }
-
}
}
}
@Test
public void testConcurrentTransactionBatchCommits() throws Exception {
- final HiveEndPoint ep = new HiveEndPoint(metaStoreURI, dbName, tblName, partitionVals);
List<WriterThd> writers = new ArrayList<WriterThd>(3);
- writers.add(new WriterThd(ep, "1,Matrix"));
- writers.add(new WriterThd(ep, "2,Gandhi"));
- writers.add(new WriterThd(ep, "3,Silence"));
+ writers.add(new WriterThd("1,Matrix"));
+ writers.add(new WriterThd("2,Gandhi"));
+ writers.add(new WriterThd("3,Silence"));
- for(WriterThd w : writers) {
+ for (WriterThd w : writers) {
w.start();
}
- for(WriterThd w : writers) {
+ for (WriterThd w : writers) {
w.join();
}
- for(WriterThd w : writers) {
- if(w.error != null) {
+ 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);
}
@@ -1376,11 +1656,11 @@ public class TestStreaming {
private ArrayList<SampleRec> 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));
+ OrcFile.readerOptions(conf).filesystem(fs));
RecordReader rows = reader.rows();
StructObjectInspector inspector = (StructObjectInspector) reader
- .getObjectInspector();
+ .getObjectInspector();
System.out.format("Found Bucket File : %s \n", orcFile.getName());
ArrayList<SampleRec> result = new ArrayList<SampleRec>();
@@ -1402,7 +1682,7 @@ public class TestStreaming {
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();
+ WritableLongObjectInspector f4ins = (WritableLongObjectInspector) fields.get(4).getFieldObjectInspector();
StructObjectInspector f5ins = (StructObjectInspector) fields.get(5).getFieldObjectInspector();
int f0 = f0ins.get(inspector.getStructFieldData(row, fields.get(0)));
@@ -1412,7 +1692,7 @@ public class TestStreaming {
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};
+ return new Object[]{f0, f1, f2, f3, f4, f5};
}
// Assumes row schema => string,int,string
@@ -1437,49 +1717,67 @@ public class TestStreaming {
// 1) Create two bucketed tables
String dbLocation = dbFolder.newFolder(dbName3).getCanonicalPath() + ".db";
- dbLocation = dbLocation.replaceAll("\\\\","/"); // for windows paths
+ 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);
+ , null, dbLocation, bucketCount);
String dbLocation2 = dbFolder.newFolder(dbName4).getCanonicalPath() + ".db";
- dbLocation2 = dbLocation2.replaceAll("\\\\","/"); // for windows paths
+ 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);
+ , null, dbLocation2, bucketCount);
// 2) Insert data into both tables
- HiveEndPoint endPt = new HiveEndPoint(metaStoreURI, db
<TRUNCATED>
[4/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/HiveEndPoint.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/HiveEndPoint.java b/streaming/src/java/org/apache/hive/streaming/HiveEndPoint.java
deleted file mode 100644
index b04e137..0000000
--- a/streaming/src/java/org/apache/hive/streaming/HiveEndPoint.java
+++ /dev/null
@@ -1,1117 +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.streaming;
-
-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 org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hive.cli.CliSessionState;
-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.LockComponentBuilder;
-import org.apache.hadoop.hive.metastore.LockRequestBuilder;
-import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.DataOperationType;
-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.NoSuchObjectException;
-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.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.session.SessionState;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hive.hcatalog.common.HCatUtil;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * 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.
- */
-public class HiveEndPoint {
- public final String metaStoreUri;
- public final String database;
- public final String table;
- public final ArrayList<String> 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<String> partitionVals) {
- this.metaStoreUri = metaStoreUri;
- if (database==null) {
- throw new IllegalArgumentException("Database cannot be null for HiveEndPoint");
- }
- this.database = database;
- this.table = table;
- if (table==null) {
- throw new IllegalArgumentException("Table cannot be null for HiveEndPoint");
- }
- this.partitionVals = partitionVals==null ? new ArrayList<String>()
- : new ArrayList<String>( partitionVals );
- }
-
-
- /**
- * @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<StreamingConnection>() {
- @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);
- }
-
- private static UserGroupInformation getUserGroupInfo(String user)
- throws ImpersonationFailed {
- try {
- return UserGroupInformation.createProxyUser(
- user, UserGroupInformation.getLoginUser());
- } catch (IOException e) {
- LOG.error("Unable to get UserGroupInfo for user : " + user, e);
- throw new ImpersonationFailed(user,e);
- }
- }
-
-
- @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<Void>() {
- @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<TransactionBatch>() {
- @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 InvalidTable, PartitionCreationFailed {
- if (ep.partitionVals.isEmpty()) {
- return;
- }
- SessionState localSession = null;
- if(SessionState.get() == null) {
- localSession = SessionState.start(new CliSessionState(conf));
- }
- IDriver driver = DriverFactory.newDriver(conf);
-
- try {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Attempting to create partition (if not existent) " + ep);
- }
-
- List<FieldSchema> partKeys = msClient.getTable(ep.database, ep.table)
- .getPartitionKeys();
- runDDL(driver, "use " + ep.database);
- String query = "alter table " + ep.table + " add if not exists partition "
- + partSpecStr(partKeys, ep.partitionVals);
- runDDL(driver, query);
- } catch (MetaException e) {
- LOG.error("Failed to create partition : " + ep, e);
- throw new PartitionCreationFailed(ep, e);
- } catch (NoSuchObjectException e) {
- LOG.error("Failed to create partition : " + ep, e);
- throw new InvalidTable(ep.database, ep.table);
- } catch (TException e) {
- LOG.error("Failed to create partition : " + ep, e);
- throw new PartitionCreationFailed(ep, e);
- } catch (QueryFailedException e) {
- LOG.error("Failed to create partition : " + ep, e);
- throw new PartitionCreationFailed(ep, e);
- } finally {
- driver.close();
- try {
- if(localSession != null) {
- localSession.close();
- }
- } catch (IOException e) {
- LOG.warn("Error closing SessionState used to run Hive DDL.");
- }
- }
- }
-
- private static boolean runDDL(IDriver driver, String sql) throws QueryFailedException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Running Hive Query: " + sql);
- }
- driver.run(sql);
- return true;
- }
-
- private static String partSpecStr(List<FieldSchema> partKeys, ArrayList<String> partVals) {
- if (partKeys.size()!=partVals.size()) {
- throw new IllegalArgumentException("Partition values:" + partVals +
- ", does not match the partition Keys in table :" + partKeys );
- }
- StringBuilder buff = new StringBuilder(partKeys.size()*20);
- buff.append(" ( ");
- int i=0;
- for (FieldSchema schema : partKeys) {
- buff.append(schema.getName());
- buff.append("='");
- buff.append(partVals.get(i));
- buff.append("'");
- if (i!=partKeys.size()-1) {
- buff.append(",");
- }
- ++i;
- }
- buff.append(" )");
- return buff.toString();
- }
-
- 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<TxnToWriteId> 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<FieldSchema> 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<Long> 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<Long> 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<Long>) ugi.doAs(new PrivilegedExceptionAction<Object>() {
- @Override
- public Object run() throws Exception {
- return msClient.openTxns(user, numTxns).getTxn_ids();
- }
- });
- }
-
- private List<TxnToWriteId> allocateWriteIdsImpl(final IMetaStoreClient msClient,
- final List<Long> txnIds, UserGroupInformation ugi)
- throws IOException, TException, InterruptedException {
- if(ugi==null) {
- return msClient.allocateTableWriteIdsBatch(txnIds, endPt.database, endPt.table);
- }
- return (List<TxnToWriteId>) ugi.doAs(new PrivilegedExceptionAction<Object>() {
- @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<Void>() {
- @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<byte[]> records)
- throws StreamingException, InterruptedException,
- ImpersonationFailed {
- checkIsClosed();
- boolean success = false;
- try {
- if (ugi == null) {
- writeImpl(records);
- } else {
- ugi.doAs(
- new PrivilegedExceptionAction<Void>() {
- @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<byte[]> 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<Void>() {
- @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<Void>() {
- @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<Void>() {
- @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)
- .setShared()
- .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
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
new file mode 100644
index 0000000..205ed6c
--- /dev/null
+++ b/streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
@@ -0,0 +1,1039 @@
+/*
+ * 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.streaming;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.classification.InterfaceStability;
+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.HiveMetaStoreUtils;
+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.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+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.Partition;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+import org.apache.hadoop.hive.ql.lockmgr.LockException;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * Streaming connection implementation for hive. To create a streaming connection, use the builder API
+ * to create record writer first followed by the connection itself. Once connection is created, clients can
+ * begin a transaction, keep writing using the connection, commit the transaction and close connection when done.
+ * To bind to the correct metastore, HiveConf object has to be created from hive-site.xml or HIVE_CONF_DIR.
+ * If hive conf is manually created, metastore uri has to be set correctly. If hive conf object is not specified,
+ * "thrift://localhost:9083" will be used as default.
+ * <br/><br/>
+ * NOTE: The streaming connection APIs and record writer APIs are not thread-safe. Streaming connection creation,
+ * begin/commit/abort transactions, write and close has to be called in the same thread. If close() or
+ * abortTransaction() has to be triggered from a separate thread it has to be co-ordinated via external variables or
+ * synchronization mechanism
+ * <br/><br/>
+ * Example usage:
+ * <pre>{@code
+ * // create delimited record writer whose schema exactly matches table schema
+ * StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ * .withFieldDelimiter(',')
+ * .build();
+ *
+ * // create and open streaming connection (default.src table has to exist already)
+ * StreamingConnection connection = HiveStreamingConnection.newBuilder()
+ * .withDatabase("default")
+ * .withTable("src")
+ * .withAgentInfo("nifi-agent")
+ * .withRecordWriter(writer)
+ * .withHiveConf(hiveConf)
+ * .connect();
+ *
+ * // begin a transaction, write records and commit 1st transaction
+ * connection.beginTransaction();
+ * connection.write("key1,val1".getBytes());
+ * connection.write("key2,val2".getBytes());
+ * connection.commitTransaction();
+ *
+ * // begin another transaction, write more records and commit 2nd transaction
+ * connection.beginTransaction();
+ * connection.write("key3,val3".getBytes());
+ * connection.write("key4,val4".getBytes());
+ * connection.commitTransaction();
+ *
+ * // close the streaming connection
+ * connection.close();
+ * }
+ * </pre>
+ */
+public class HiveStreamingConnection implements StreamingConnection {
+ private static final Logger LOG = LoggerFactory.getLogger(HiveStreamingConnection.class.getName());
+
+ private static final String DEFAULT_METASTORE_URI = "thrift://localhost:9083";
+ private static final int DEFAULT_TRANSACTION_BATCH_SIZE = 1;
+ private static final int DEFAULT_HEARTBEAT_INTERVAL = 60 * 1000;
+ private static final boolean DEFAULT_STREAMING_OPTIMIZATIONS_ENABLED = true;
+
+ public 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;
+ }
+ }
+
+ // fields populated from builder
+ private String database;
+ private String table;
+ private List<String> staticPartitionValues;
+ private String agentInfo;
+ private int transactionBatchSize;
+ private RecordWriter recordWriter;
+ private TransactionBatch currentTransactionBatch;
+ private HiveConf conf;
+ private boolean streamingOptimizations;
+ private AtomicBoolean isConnectionClosed = new AtomicBoolean(false);
+
+ // internal fields
+ private boolean isPartitionedTable;
+ private IMetaStoreClient msClient;
+ private IMetaStoreClient heartbeatMSClient;
+ private final String username;
+ private final boolean secureMode;
+ private Table tableObject = null;
+ private String metastoreUri;
+
+ private HiveStreamingConnection(Builder builder) throws StreamingException {
+ this.database = builder.database.toLowerCase();
+ this.table = builder.table.toLowerCase();
+ this.staticPartitionValues = builder.staticPartitionValues;
+ this.conf = builder.hiveConf;
+ this.agentInfo = builder.agentInfo;
+ this.streamingOptimizations = builder.streamingOptimizations;
+ UserGroupInformation loggedInUser = null;
+ try {
+ loggedInUser = UserGroupInformation.getLoginUser();
+ } catch (IOException e) {
+ LOG.warn("Unable to get logged in user via UGI. err: {}", e.getMessage());
+ }
+ if (loggedInUser == null) {
+ this.username = System.getProperty("user.name");
+ this.secureMode = false;
+ } else {
+ this.username = loggedInUser.getShortUserName();
+ this.secureMode = loggedInUser.hasKerberosCredentials();
+ }
+ this.transactionBatchSize = builder.transactionBatchSize;
+ this.recordWriter = builder.recordWriter;
+ if (agentInfo == null) {
+ try {
+ agentInfo = username + ":" + InetAddress.getLocalHost().getHostName() + ":" + Thread.currentThread().getName();
+ } catch (UnknownHostException e) {
+ // ignore and use UUID instead
+ this.agentInfo = UUID.randomUUID().toString();
+ }
+ }
+ if (conf == null) {
+ conf = createHiveConf(this.getClass(), DEFAULT_METASTORE_URI);
+ }
+ overrideConfSettings(conf);
+ this.metastoreUri = conf.get(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName());
+ this.msClient = getMetaStoreClient(conf, metastoreUri, 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.heartbeatMSClient = getMetaStoreClient(conf, metastoreUri, secureMode);
+ validateTable();
+ LOG.info("STREAMING CONNECTION INFO: {}", toConnectionInfoString());
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ public static class Builder {
+ private String database;
+ private String table;
+ private List<String> staticPartitionValues;
+ private String agentInfo;
+ private HiveConf hiveConf;
+ private int transactionBatchSize = DEFAULT_TRANSACTION_BATCH_SIZE;
+ private boolean streamingOptimizations = DEFAULT_STREAMING_OPTIMIZATIONS_ENABLED;
+ private RecordWriter recordWriter;
+
+ /**
+ * Specify database to use for streaming connection.
+ *
+ * @param database - db name
+ * @return - builder
+ */
+ public Builder withDatabase(final String database) {
+ this.database = database;
+ return this;
+ }
+
+ /**
+ * Specify table to use for streaming connection.
+ *
+ * @param table - table name
+ * @return - builder
+ */
+ public Builder withTable(final String table) {
+ this.table = table;
+ return this;
+ }
+
+ /**
+ * Specify the name of partition to use for streaming connection.
+ *
+ * @param staticPartitionValues - static partition values
+ * @return - builder
+ */
+ public Builder withStaticPartitionValues(final List<String> staticPartitionValues) {
+ this.staticPartitionValues = staticPartitionValues == null ? null : new ArrayList<>(staticPartitionValues);
+ return this;
+ }
+
+ /**
+ * Specify agent info to use for streaming connection.
+ *
+ * @param agentInfo - agent info
+ * @return - builder
+ */
+ public Builder withAgentInfo(final String agentInfo) {
+ this.agentInfo = agentInfo;
+ return this;
+ }
+
+ /**
+ * Specify hive configuration object to use for streaming connection.
+ * Generate this object by point to already existing hive-site.xml or HIVE_CONF_DIR.
+ * Make sure if metastore URI has been set correctly else thrift://localhost:9083 will be
+ * used as default.
+ *
+ * @param hiveConf - hive conf object
+ * @return - builder
+ */
+ public Builder withHiveConf(final HiveConf hiveConf) {
+ this.hiveConf = hiveConf;
+ return this;
+ }
+
+ /**
+ * Transaction batch size to use (default value is 10). This is expert level configuration.
+ * For every transaction batch a delta directory will be created which will impact
+ * when compaction will trigger.
+ * NOTE: This is evolving API and is subject to change/might not be honored in future releases.
+ *
+ * @param transactionBatchSize - transaction batch size
+ * @return - builder
+ */
+ @InterfaceStability.Evolving
+ public Builder withTransactionBatchSize(final int transactionBatchSize) {
+ this.transactionBatchSize = transactionBatchSize;
+ return this;
+ }
+
+ /**
+ * Whether to enable streaming optimizations. This is expert level configurations.
+ * Disabling streaming optimizations will have significant impact to performance and memory consumption.
+ *
+ * @param enable - flag to enable or not
+ * @return - builder
+ */
+ public Builder withStreamingOptimizations(final boolean enable) {
+ this.streamingOptimizations = enable;
+ return this;
+ }
+
+ /**
+ * Record writer to use for writing records to destination table.
+ *
+ * @param recordWriter - record writer
+ * @return - builder
+ */
+ public Builder withRecordWriter(final RecordWriter recordWriter) {
+ this.recordWriter = recordWriter;
+ return this;
+ }
+
+ /**
+ * Returning a streaming connection to hive.
+ *
+ * @return - hive streaming connection
+ */
+ public HiveStreamingConnection connect() throws StreamingException {
+ if (database == null) {
+ throw new StreamingException("Database cannot be null for streaming connection");
+ }
+ if (table == null) {
+ throw new StreamingException("Table cannot be null for streaming connection");
+ }
+ if (recordWriter == null) {
+ throw new StreamingException("Record writer cannot be null for streaming connection");
+ }
+ return new HiveStreamingConnection(this);
+ }
+ }
+
+ private void setPartitionedTable(boolean isPartitionedTable) {
+ this.isPartitionedTable = isPartitionedTable;
+ }
+
+ @Override
+ public String toString() {
+ return "{ metaStoreUri: " + metastoreUri + ", database: " + database + ", table: " + table + " }";
+ }
+
+ private String toConnectionInfoString() {
+ return "{ metastore-uri: " + metastoreUri + ", " +
+ "database: " + database + ", " +
+ "table: " + table + ", " +
+ "partitioned-table: " + isPartitionedTable() + ", " +
+ "dynamic-partitioning: " + isDynamicPartitioning() + ", " +
+ "username: " + username + ", " +
+ "secure-mode: " + secureMode + ", " +
+ "record-writer: " + recordWriter.getClass().getSimpleName() + ", " +
+ "agent-info: " + agentInfo + " }";
+ }
+
+ @VisibleForTesting
+ String toTransactionString() {
+ return currentTransactionBatch == null ? "" : currentTransactionBatch.toString();
+ }
+
+ @Override
+ public PartitionInfo createPartitionIfNotExists(final List<String> partitionValues) throws StreamingException {
+ String partLocation = null;
+ String partName = null;
+ boolean exists = false;
+ try {
+ Map<String, String> partSpec = Warehouse.makeSpecFromValues(tableObject.getPartitionKeys(), partitionValues);
+ AddPartitionDesc addPartitionDesc = new AddPartitionDesc(database, table, true);
+ partName = Warehouse.makePartName(tableObject.getPartitionKeys(), partitionValues);
+ partLocation = new Path(tableObject.getDataLocation(), Warehouse.makePartPath(partSpec)).toString();
+ addPartitionDesc.addPartition(partSpec, partLocation);
+ Partition partition = Hive.convertAddSpecToMetaPartition(tableObject, addPartitionDesc.getPartition(0), conf);
+ msClient.add_partition(partition);
+ } catch (AlreadyExistsException e) {
+ exists = true;
+ } catch (HiveException | TException e) {
+ throw new StreamingException("Unable to creation partition for values: " + partitionValues + " connection: " +
+ toConnectionInfoString());
+ }
+ return new PartitionInfo(partName, partLocation, exists);
+ }
+
+ private void validateTable() throws InvalidTable, ConnectionError {
+ try {
+ tableObject = new Table(msClient.getTable(database, table));
+ } catch (Exception e) {
+ LOG.warn("Unable to validate the table for connection: " + toConnectionInfoString(), e);
+ throw new InvalidTable(database, table, e);
+ }
+ // 1 - check that the table is Acid
+ if (!AcidUtils.isFullAcidTable(tableObject)) {
+ LOG.error("HiveEndPoint " + this + " must use an acid table");
+ throw new InvalidTable(database, table, "is not an Acid table");
+ }
+
+ if (tableObject.getPartitionKeys() != null && !tableObject.getPartitionKeys().isEmpty()) {
+ setPartitionedTable(true);
+ } else {
+ setPartitionedTable(false);
+ }
+
+ // partition values are specified on non-partitioned table
+ if (!isPartitionedTable() && (staticPartitionValues != null && !staticPartitionValues.isEmpty())) {
+ // Invalid if table is not partitioned, but endPoint's partitionVals is not empty
+ String errMsg = this.toString() + " specifies partitions for un-partitioned table";
+ LOG.error(errMsg);
+ throw new ConnectionError(errMsg);
+ }
+ }
+
+ private static class HeartbeatRunnable implements Runnable {
+ private final IMetaStoreClient heartbeatMSClient;
+ private final AtomicLong minTxnId;
+ private final long maxTxnId;
+ private final ReentrantLock transactionLock;
+ private final AtomicBoolean isTxnClosed;
+
+ HeartbeatRunnable(final IMetaStoreClient heartbeatMSClient, final AtomicLong minTxnId, final long maxTxnId,
+ final ReentrantLock transactionLock, final AtomicBoolean isTxnClosed) {
+ this.heartbeatMSClient = heartbeatMSClient;
+ this.minTxnId = minTxnId;
+ this.maxTxnId = maxTxnId;
+ this.transactionLock = transactionLock;
+ this.isTxnClosed = isTxnClosed;
+ }
+
+ @Override
+ public void run() {
+ transactionLock.lock();
+ try {
+ if (minTxnId.get() > 0) {
+ HeartbeatTxnRangeResponse resp = heartbeatMSClient.heartbeatTxnRange(minTxnId.get(), maxTxnId);
+ if (!resp.getAborted().isEmpty() || !resp.getNosuch().isEmpty()) {
+ LOG.error("Heartbeat failure: {}", resp.toString());
+ isTxnClosed.set(true);
+ } else {
+ LOG.info("Heartbeat sent for range: [{}-{}]", minTxnId.get(), maxTxnId);
+ }
+ }
+ } catch (TException e) {
+ LOG.warn("Failure to heartbeat for transaction range: [" + minTxnId.get() + "-" + maxTxnId + "]", e);
+ } finally {
+ transactionLock.unlock();
+ }
+ }
+ }
+
+ private void beginNextTransaction() throws StreamingException {
+ if (currentTransactionBatch == null) {
+ currentTransactionBatch = createNewTransactionBatch();
+ LOG.info("Opened new transaction batch {}", currentTransactionBatch);
+ }
+
+ if (currentTransactionBatch.isClosed()) {
+ throw new IllegalStateException("Cannot begin next transaction on a closed streaming connection");
+ }
+
+ if (currentTransactionBatch.remainingTransactions() == 0) {
+ LOG.info("Transaction batch {} is done. Rolling over to next transaction batch.",
+ currentTransactionBatch);
+ currentTransactionBatch.close();
+ currentTransactionBatch = createNewTransactionBatch();
+ LOG.info("Rolled over to new transaction batch {}", currentTransactionBatch);
+ }
+ currentTransactionBatch.beginNextTransaction();
+ }
+
+ private TransactionBatch createNewTransactionBatch() throws StreamingException {
+ return new TransactionBatch(this);
+ }
+
+ private void checkClosedState() throws StreamingException {
+ if (isConnectionClosed.get()) {
+ throw new StreamingException("Streaming connection is closed already.");
+ }
+ }
+
+ private void checkState() throws StreamingException {
+ checkClosedState();
+ if (currentTransactionBatch == null) {
+ throw new StreamingException("Transaction batch is null. Missing beginTransaction?");
+ }
+ if (currentTransactionBatch.state != TxnState.OPEN) {
+ throw new StreamingException("Transaction state is not OPEN. Missing beginTransaction?");
+ }
+ }
+
+ @Override
+ public void write(final byte[] record) throws StreamingException {
+ checkState();
+ currentTransactionBatch.write(record);
+ }
+
+ @Override
+ public void beginTransaction() throws StreamingException {
+ checkClosedState();
+ beginNextTransaction();
+ }
+
+ @Override
+ public void commitTransaction() throws StreamingException {
+ checkState();
+ currentTransactionBatch.commit();
+ }
+
+ @Override
+ public void abortTransaction() throws StreamingException {
+ checkState();
+ currentTransactionBatch.abort();
+ }
+
+ @Override
+ public void close() {
+ if (isConnectionClosed.get()) {
+ return;
+ }
+ isConnectionClosed.set(true);
+ try {
+ if (currentTransactionBatch != null) {
+ currentTransactionBatch.close();
+ }
+ } catch (StreamingException e) {
+ LOG.error("Unable to close current transaction batch: " + currentTransactionBatch, e);
+ } finally {
+ msClient.close();
+ heartbeatMSClient.close();
+ }
+ }
+
+ private static IMetaStoreClient getMetaStoreClient(HiveConf conf, String metastoreUri, boolean secureMode)
+ throws ConnectionError {
+ if (metastoreUri != null) {
+ conf.set(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName(), metastoreUri);
+ }
+ if (secureMode) {
+ conf.setBoolean(MetastoreConf.ConfVars.USE_THRIFT_SASL.getHiveName(), true);
+ }
+ try {
+ return HiveMetaStoreUtils.getHiveMetastoreClient(conf);
+ } catch (MetaException | IOException e) {
+ throw new ConnectionError("Error connecting to Hive Metastore URI: "
+ + metastoreUri + ". " + e.getMessage(), e);
+ }
+ }
+
+ @VisibleForTesting
+ TxnState getCurrentTransactionState() {
+ return currentTransactionBatch.getCurrentTransactionState();
+ }
+
+ @VisibleForTesting
+ int remainingTransactions() {
+ return currentTransactionBatch.remainingTransactions();
+ }
+
+ @VisibleForTesting
+ Long getCurrentTxnId() {
+ return currentTransactionBatch.getCurrentTxnId();
+ }
+
+ private static class TransactionBatch {
+ private String username;
+ private HiveStreamingConnection conn;
+ private IMetaStoreClient msClient;
+ private IMetaStoreClient heartbeatMSClient;
+ private ScheduledExecutorService scheduledExecutorService;
+ private RecordWriter recordWriter;
+ private String partNameForLock = null;
+ private List<TxnToWriteId> txnToWriteIds;
+ private int currentTxnIndex = -1;
+ private TxnState state;
+ private LockRequest lockRequest = null;
+ // heartbeats can only be sent for open transactions.
+ // there is a race between committing/aborting a transaction and heartbeat.
+ // Example: If a heartbeat is sent for committed txn, exception will be thrown.
+ // Similarly if we don't send a heartbeat, metastore server might abort a txn
+ // for missed heartbeat right before commit txn call.
+ // This lock is used to mutex commit/abort and heartbeat calls
+ private final ReentrantLock transactionLock = new ReentrantLock();
+ // min txn id is incremented linearly within a transaction batch.
+ // keeping minTxnId atomic as it is shared with heartbeat thread
+ private final AtomicLong minTxnId;
+ // max txn id does not change for a transaction batch
+ private final long maxTxnId;
+
+ /**
+ * 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 final AtomicBoolean isTxnClosed = new AtomicBoolean(false);
+ private String agentInfo;
+ private int numTxns;
+ /**
+ * Tracks the state of each transaction
+ */
+ private TxnState[] txnStatus;
+ /**
+ * ID of the last txn used by {@link #beginNextTransactionImpl()}
+ */
+ private long lastTxnUsed;
+
+ /**
+ * Represents a batch of transactions acquired from MetaStore
+ *
+ * @param conn - hive streaming connection
+ * @throws StreamingException if failed to create new RecordUpdater for batch
+ */
+ private TransactionBatch(HiveStreamingConnection conn) throws StreamingException {
+ boolean success = false;
+ try {
+ if (conn.isPartitionedTable() && !conn.isDynamicPartitioning()) {
+ List<FieldSchema> partKeys = conn.tableObject.getPartitionKeys();
+ partNameForLock = Warehouse.makePartName(partKeys, conn.staticPartitionValues);
+ }
+ this.conn = conn;
+ this.username = conn.username;
+ this.msClient = conn.msClient;
+ this.heartbeatMSClient = conn.heartbeatMSClient;
+ this.recordWriter = conn.recordWriter;
+ this.agentInfo = conn.agentInfo;
+ this.numTxns = conn.transactionBatchSize;
+
+ setupHeartBeatThread();
+
+ List<Long> txnIds = openTxnImpl(msClient, username, numTxns);
+ txnToWriteIds = allocateWriteIdsImpl(msClient, txnIds);
+ 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;
+
+ // initialize record writer with connection and write id info
+ recordWriter.init(conn, txnToWriteIds.get(0).getWriteId(), txnToWriteIds.get(numTxns - 1).getWriteId());
+ this.minTxnId = new AtomicLong(txnIds.get(0));
+ this.maxTxnId = txnIds.get(txnIds.size() - 1);
+ success = true;
+ } catch (TException e) {
+ throw new StreamingException(conn.toString(), e);
+ } finally {
+ //clean up if above throws
+ markDead(success);
+ }
+ }
+
+ private void setupHeartBeatThread() {
+ // start heartbeat thread
+ ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+ .setNameFormat("HiveStreamingConnection-Heartbeat-Thread")
+ .build();
+ this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(threadFactory);
+ long heartBeatInterval;
+ long initialDelay;
+ try {
+ // if HIVE_TXN_TIMEOUT is defined, heartbeat interval will be HIVE_TXN_TIMEOUT/2
+ heartBeatInterval = DbTxnManager.getHeartbeatInterval(conn.conf);
+ } catch (LockException e) {
+ heartBeatInterval = DEFAULT_HEARTBEAT_INTERVAL;
+ }
+ // to introduce some randomness and to avoid hammering the metastore at the same time (same logic as DbTxnManager)
+ initialDelay = (long) (heartBeatInterval * 0.75 * Math.random());
+ LOG.info("Starting heartbeat thread with interval: {} ms initialDelay: {} ms for agentInfo: {}",
+ heartBeatInterval, initialDelay, conn.agentInfo);
+ Runnable runnable = new HeartbeatRunnable(heartbeatMSClient, minTxnId, maxTxnId, transactionLock, isTxnClosed);
+ this.scheduledExecutorService.scheduleWithFixedDelay(runnable, initialDelay, heartBeatInterval, TimeUnit
+ .MILLISECONDS);
+ }
+
+ private List<Long> openTxnImpl(final IMetaStoreClient msClient, final String user, final int numTxns)
+ throws TException {
+ return msClient.openTxns(user, numTxns).getTxn_ids();
+ }
+
+ private List<TxnToWriteId> allocateWriteIdsImpl(final IMetaStoreClient msClient,
+ final List<Long> txnIds) throws TException {
+ return msClient.allocateTableWriteIdsBatch(txnIds, conn.database, conn.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 connection = " + conn + "; " + sb;
+ }
+
+ private void beginNextTransaction() throws StreamingException {
+ checkIsClosed();
+ beginNextTransactionImpl();
+ }
+
+ private void beginNextTransactionImpl() throws TransactionError {
+ state = TxnState.INACTIVE;//clear state from previous txn
+ if ((currentTxnIndex + 1) >= txnToWriteIds.size()) {
+ throw new InvalidTransactionState("No more transactions available in" +
+ " next batch for connection: " + conn + " user: " + username);
+ }
+ currentTxnIndex++;
+ state = TxnState.OPEN;
+ lastTxnUsed = getCurrentTxnId();
+ lockRequest = createLockRequest(conn, partNameForLock, username, getCurrentTxnId(), agentInfo);
+ try {
+ LockResponse res = msClient.lock(lockRequest);
+ if (res.getState() != LockState.ACQUIRED) {
+ throw new TransactionError("Unable to acquire lock on " + conn);
+ }
+ } catch (TException e) {
+ throw new TransactionError("Unable to acquire lock on " + conn, e);
+ }
+ }
+
+ long getCurrentTxnId() {
+ if (currentTxnIndex >= 0) {
+ return txnToWriteIds.get(currentTxnIndex).getTxnId();
+ }
+ return -1L;
+ }
+
+ long getCurrentWriteId() {
+ if (currentTxnIndex >= 0) {
+ return txnToWriteIds.get(currentTxnIndex).getWriteId();
+ }
+ return -1L;
+ }
+
+ TxnState getCurrentTransactionState() {
+ return state;
+ }
+
+ int remainingTransactions() {
+ if (currentTxnIndex >= 0) {
+ return txnToWriteIds.size() - currentTxnIndex - 1;
+ }
+ return txnToWriteIds.size();
+ }
+
+
+ public void write(final byte[] record) throws StreamingException {
+ checkIsClosed();
+ boolean success = false;
+ try {
+ recordWriter.write(getCurrentWriteId(), record);
+ 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;
+ } finally {
+ markDead(success);
+ }
+ }
+
+ private void checkIsClosed() throws StreamingException {
+ if (isTxnClosed.get()) {
+ throw new StreamingException("Transaction" + toString() + " is 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;
+ }
+ isTxnClosed.set(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);
+ }
+ }
+
+
+ void commit() throws StreamingException {
+ checkIsClosed();
+ boolean success = false;
+ try {
+ commitImpl();
+ success = true;
+ } finally {
+ markDead(success);
+ }
+ }
+
+ private void commitImpl() throws StreamingException {
+ try {
+ recordWriter.flush();
+ TxnToWriteId txnToWriteId = txnToWriteIds.get(currentTxnIndex);
+ if (conn.isDynamicPartitioning()) {
+ List<String> partNames = new ArrayList<>(recordWriter.getPartitions());
+ msClient.addDynamicPartitions(txnToWriteId.getTxnId(), txnToWriteId.getWriteId(), conn.database, conn.table,
+ partNames, DataOperationType.INSERT);
+ }
+ transactionLock.lock();
+ try {
+ msClient.commitTxn(txnToWriteId.getTxnId());
+ // increment the min txn id so that heartbeat thread will heartbeat only from the next open transaction.
+ // the current transaction is going to committed or fail, so don't need heartbeat for current transaction.
+ if (currentTxnIndex + 1 < txnToWriteIds.size()) {
+ minTxnId.set(txnToWriteIds.get(currentTxnIndex + 1).getTxnId());
+ } else {
+ // exhausted the batch, no longer have to heartbeat for current txn batch
+ minTxnId.set(-1);
+ }
+ } finally {
+ transactionLock.unlock();
+ }
+ 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 commitTransaction transaction"
+ + getCurrentTxnId(), e);
+ }
+ }
+
+ void abort() throws StreamingException {
+ if (isTxnClosed.get()) {
+ /*
+ * 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 abortTransaction()
+ * error doesn't get misleading errors
+ */
+ return;
+ }
+ abort(false);
+ }
+
+ private void abort(final boolean abortAllRemaining) throws StreamingException {
+ abortImpl(abortAllRemaining);
+ }
+
+ private void abortImpl(boolean abortAllRemaining) throws StreamingException {
+ transactionLock.lock();
+ try {
+ if (abortAllRemaining) {
+ // we are aborting all txns in the current batch, so no need to heartbeat
+ minTxnId.set(-1);
+ //when last txn finished (abortTransaction/commitTransaction) 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 {
+ // we are aborting only the current transaction, so move the min range for heartbeat or disable heartbeat
+ // if the current txn is last in the batch.
+ if (currentTxnIndex + 1 < txnToWriteIds.size()) {
+ minTxnId.set(txnToWriteIds.get(currentTxnIndex + 1).getTxnId());
+ } else {
+ // exhausted the batch, no longer have to heartbeat
+ minTxnId.set(-1);
+ }
+ long currTxnId = getCurrentTxnId();
+ if (currTxnId > 0) {
+ msClient.rollbackTxn(currTxnId);
+ txnStatus[currentTxnIndex] = TxnState.ABORTED;
+ }
+ }
+ state = TxnState.ABORTED;
+ } 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);
+ } finally {
+ transactionLock.unlock();
+ }
+ }
+
+ public boolean isClosed() {
+ return isTxnClosed.get();
+ }
+
+ /**
+ * Close the TransactionBatch. This will abort any still open txns in this batch.
+ *
+ * @throws StreamingException - failure when closing transaction batch
+ */
+ public void close() throws StreamingException {
+ if (isTxnClosed.get()) {
+ return;
+ }
+ isTxnClosed.set(true);
+ abortImpl(true);
+ closeImpl();
+ }
+
+ private void closeImpl() throws StreamingException {
+ state = TxnState.INACTIVE;
+ recordWriter.close();
+ if (scheduledExecutorService != null) {
+ scheduledExecutorService.shutdownNow();
+ }
+ }
+
+ static LockRequest createLockRequest(final HiveStreamingConnection connection,
+ String partNameForLock, String user, long txnId, String agentInfo) {
+ LockRequestBuilder requestBuilder = new LockRequestBuilder(agentInfo);
+ requestBuilder.setUser(user);
+ requestBuilder.setTransactionId(txnId);
+
+ LockComponentBuilder lockCompBuilder = new LockComponentBuilder()
+ .setDbName(connection.database)
+ .setTableName(connection.table)
+ .setShared()
+ .setOperationType(DataOperationType.INSERT);
+ if (connection.isDynamicPartitioning()) {
+ lockCompBuilder.setIsDynamicPartitionWrite(true);
+ }
+ if (partNameForLock != null && !partNameForLock.isEmpty()) {
+ lockCompBuilder.setPartitionName(partNameForLock);
+ }
+ requestBuilder.addLockComponent(lockCompBuilder.build());
+
+ return requestBuilder.build();
+ }
+ }
+
+ private HiveConf createHiveConf(Class<?> clazz, String metaStoreUri) {
+ HiveConf conf = new HiveConf(clazz);
+ if (metaStoreUri != null) {
+ conf.set(MetastoreConf.ConfVars.THRIFT_URIS.getHiveName(), metaStoreUri);
+ }
+ return conf;
+ }
+
+ private void overrideConfSettings(HiveConf conf) {
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName());
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+ setHiveConf(conf, MetastoreConf.ConfVars.EXECUTE_SET_UGI.getHiveName());
+ // Avoids creating Tez Client sessions internally as it takes much longer currently
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr");
+ setHiveConf(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
+ if (streamingOptimizations) {
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_ORC_DELTA_STREAMING_OPTIMIZATIONS_ENABLED, true);
+ }
+ // since same thread creates metastore client for streaming connection thread and heartbeat thread we explicitly
+ // disable metastore client cache
+ setHiveConf(conf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_ENABLED, false);
+ }
+
+ 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, true);
+ }
+
+ private static void setHiveConf(HiveConf conf, String var) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Overriding HiveConf setting : " + var + " = " + true);
+ }
+ conf.setBoolean(var, true);
+ }
+
+ @Override
+ public HiveConf getHiveConf() {
+ return conf;
+ }
+
+ @Override
+ public String getMetastoreUri() {
+ return metastoreUri;
+ }
+
+ @Override
+ public String getDatabase() {
+ return database;
+ }
+
+ @Override
+ public String getTable() {
+ return table;
+ }
+
+ @Override
+ public List<String> getStaticPartitionValues() {
+ return staticPartitionValues;
+ }
+
+ @Override
+ public String getAgentInfo() {
+ return agentInfo;
+ }
+
+ @Override
+ public boolean isPartitionedTable() {
+ return isPartitionedTable;
+ }
+
+ @Override
+ public boolean isDynamicPartitioning() {
+ return isPartitionedTable() && (staticPartitionValues == null || staticPartitionValues.isEmpty());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/streaming/src/java/org/apache/hive/streaming/ImpersonationFailed.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/ImpersonationFailed.java b/streaming/src/java/org/apache/hive/streaming/ImpersonationFailed.java
deleted file mode 100644
index 23e17e7..0000000
--- a/streaming/src/java/org/apache/hive/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.streaming;
-
-public class ImpersonationFailed extends StreamingException {
- public ImpersonationFailed(String username, Exception e) {
- super("Failed to impersonate user " + username, e);
- }
-}
[6/7] hive git commit: HIVE-19211: New streaming ingest API and
support for dynamic partitioning (Prasanth Jayachandran reviewed by Eugene
Koifman)
Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
----------------------------------------------------------------------
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 8ee033d..7e17d5d 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
@@ -17,12 +17,16 @@
*/
package org.apache.hadoop.hive.ql.txn.compactor;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileWriter;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Random;
@@ -75,11 +79,13 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
import org.apache.hadoop.hive.ql.session.SessionState;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hive.hcatalog.common.HCatUtil;
-import org.apache.hive.streaming.DelimitedInputWriter;
-import org.apache.hive.streaming.HiveEndPoint;
+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;
-import org.apache.hive.streaming.TransactionBatch;
+import org.apache.hive.streaming.StrictDelimitedInputWriter;
import org.apache.orc.OrcConf;
import org.junit.After;
import org.junit.Assert;
@@ -87,19 +93,32 @@ import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/**
- */
+@RunWith(Parameterized.class)
public class TestCompactor {
private static final AtomicInteger salt = new AtomicInteger(new Random().nextInt());
private static final Logger LOG = LoggerFactory.getLogger(TestCompactor.class);
private final String TEST_DATA_DIR = HCatUtil.makePathASafeFileName(System.getProperty("java.io.tmpdir") +
- File.separator + TestCompactor.class.getCanonicalName() + "-" + System.currentTimeMillis() + "_" + salt.getAndIncrement());
+ File.separator + TestCompactor.class.getCanonicalName() + "-" + System.currentTimeMillis() + "_" +
+ salt.getAndIncrement());
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<Object[]> data() {
+ return Arrays.asList(new Object[][]{{true}, {false}});
+ }
+
+ private boolean newStreamingAPI;
+
+ public TestCompactor(boolean newStreamingAPI) {
+ this.newStreamingAPI = newStreamingAPI;
+ }
+
@Rule
public TemporaryFolder stagingFolder = new TemporaryFolder();
private HiveConf conf;
@@ -113,7 +132,7 @@ public class TestCompactor {
if (f.exists()) {
FileUtil.fullyDelete(f);
}
- if(!(new File(TEST_WAREHOUSE_DIR).mkdirs())) {
+ if (!(new File(TEST_WAREHOUSE_DIR).mkdirs())) {
throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR);
}
@@ -147,25 +166,26 @@ public class TestCompactor {
}
createTestDataFile(BASIC_FILE_NAME, input);
}
+
@After
public void tearDown() {
conf = null;
- if(msClient != null) {
+ if (msClient != null) {
msClient.close();
}
- if(driver != null) {
+ if (driver != null) {
driver.close();
}
}
/**
* Simple schema evolution add columns with partitioning.
+ *
* @throws Exception
*/
@Test
public void schemaEvolutionAddColDynamicPartitioningInsert() throws Exception {
String tblName = "dpct";
- List<String> colNames = Arrays.asList("a", "b");
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
" PARTITIONED BY(ds string)" +
@@ -174,7 +194,7 @@ public class TestCompactor {
// First INSERT round.
executeStatementOnDriver("insert into " + tblName + " partition (ds) values (1, 'fred', " +
- "'today'), (2, 'wilma', 'yesterday')", driver);
+ "'today'), (2, 'wilma', 'yesterday')", driver);
// ALTER TABLE ... ADD COLUMNS
executeStatementOnDriver("ALTER TABLE " + tblName + " ADD COLUMNS(c int)", driver);
@@ -191,7 +211,7 @@ public class TestCompactor {
executeStatementOnDriver("insert into " + tblName + " partition (ds) values " +
"(3, 'mark', 1900, 'soon'), (4, 'douglas', 1901, 'last_century'), " +
"(5, 'doc', 1902, 'yesterday')",
- driver);
+ driver);
// Validate there the new insertions for column c.
executeStatementOnDriver("SELECT * FROM " + tblName + " ORDER BY a", driver);
@@ -205,7 +225,7 @@ public class TestCompactor {
Assert.assertEquals("5\tdoc\t1902\tyesterday", valuesReadFromHiveDriver.get(4));
Initiator initiator = new Initiator();
- initiator.setThreadId((int)initiator.getId());
+ initiator.setThreadId((int) initiator.getId());
conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 0);
initiator.setConf(conf);
AtomicBoolean stop = new AtomicBoolean();
@@ -246,14 +266,13 @@ public class TestCompactor {
@Test
public void schemaEvolutionAddColDynamicPartitioningUpdate() throws Exception {
String tblName = "udpct";
- List<String> colNames = Arrays.asList("a", "b");
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
" PARTITIONED BY(ds string)" +
" CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
" STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
executeStatementOnDriver("insert into " + tblName + " partition (ds) values (1, 'fred', " +
- "'today'), (2, 'wilma', 'yesterday')", driver);
+ "'today'), (2, 'wilma', 'yesterday')", driver);
executeStatementOnDriver("update " + tblName + " set b = 'barney'", driver);
@@ -280,7 +299,7 @@ public class TestCompactor {
executeStatementOnDriver("insert into " + tblName + " partition (ds) values " +
"(3, 'mark', 1900, 'soon'), (4, 'douglas', 1901, 'last_century'), " +
"(5, 'doc', 1902, 'yesterday')",
- driver);
+ driver);
// Validate there the new insertions for column c.
executeStatementOnDriver("SELECT * FROM " + tblName + " ORDER BY a", driver);
@@ -307,7 +326,7 @@ public class TestCompactor {
Assert.assertEquals("5\tdoc\t2000\tyesterday", valuesReadFromHiveDriver.get(4));
Initiator initiator = new Initiator();
- initiator.setThreadId((int)initiator.getId());
+ initiator.setThreadId((int) initiator.getId());
// Set to 1 so insert doesn't set it off but update does
conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 1);
initiator.setConf(conf);
@@ -354,10 +373,10 @@ public class TestCompactor {
* 4. insert some data into the table using StreamingAPI
* 5. Trigger major compaction (which should update stats)
* 6. check that stats have been updated
- * @throws Exception
- * todo:
- * 2. add non-partitioned test
- * 4. add a test with sorted table?
+ *
+ * @throws Exception todo:
+ * 2. add non-partitioned test
+ * 4. add a test with sorted table?
*/
@Test
public void testStatsAfterCompactionPartTbl() throws Exception {
@@ -407,7 +426,7 @@ public class TestCompactor {
Map<String, List<ColumnStatisticsObj>> stats = msClient.getPartitionColumnStatistics(ci.dbname,
ci.tableName, Arrays.asList(ci.partName), colNames);
List<ColumnStatisticsObj> colStats = stats.get(ci.partName);
- Assert.assertNotNull("No stats found for partition " + ci.partName, colStats);
+ assertNotNull("No stats found for partition " + ci.partName, colStats);
Assert.assertEquals("Expected column 'a' at index 0", "a", colStats.get(0).getColName());
Assert.assertEquals("Expected column 'b' at index 1", "b", colStats.get(1).getColName());
LongColumnStatsData colAStats = colStats.get(0).getStatsData().getLongStats();
@@ -428,32 +447,58 @@ public class TestCompactor {
LongColumnStatsData colAStatsPart2 = colStats.get(0).getStatsData().getLongStats();
StringColumnStatsData colBStatsPart2 = colStats.get(1).getStatsData().getStringStats();
-
- HiveEndPoint endPt = new HiveEndPoint(null, ci.dbname, ci.tableName, Arrays.asList("0"));
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
+ 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*/
- StreamingConnection connection = endPt.newConnection(true, "UT_" + Thread.currentThread().getName());
+ 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();
+ 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.beginNextTransaction();
+ txnBatch.write("52,Tel Aviv".getBytes());
+ txnBatch.write("53,Atlantis".getBytes());
+ txnBatch.write("53,Boston".getBytes());
+ txnBatch.commit();
- txnBatch.close();
- connection.close();
+ txnBatch.close();
+ 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
@@ -478,7 +523,7 @@ public class TestCompactor {
stats = msClient.getPartitionColumnStatistics(ci.dbname, ci.tableName,
Arrays.asList(ci.partName), colNames);
colStats = stats.get(ci.partName);
- Assert.assertNotNull("No stats found for partition " + ci.partName, colStats);
+ assertNotNull("No stats found for partition " + ci.partName, colStats);
Assert.assertEquals("Expected column 'a' at index 0", "a", colStats.get(0).getColName());
Assert.assertEquals("Expected column 'b' at index 1", "b", colStats.get(1).getColName());
colAStats = colStats.get(0).getStatsData().getLongStats();
@@ -489,7 +534,7 @@ public class TestCompactor {
colBStats = colStats.get(1).getStatsData().getStringStats();
Assert.assertEquals("maxColLen b", 14, colBStats.getMaxColLen());
//cast it to long to get rid of periodic decimal
- Assert.assertEquals("avgColLen b", (long)6.1111111111, (long)colBStats.getAvgColLen());
+ Assert.assertEquals("avgColLen b", (long) 6.1111111111, (long) colBStats.getAvgColLen());
Assert.assertEquals("numNulls b", 0, colBStats.getNumNulls());
Assert.assertEquals("nunDVs", 8, colBStats.getNumDVs());
@@ -506,17 +551,16 @@ public class TestCompactor {
@Test
public void dynamicPartitioningInsert() throws Exception {
String tblName = "dpct";
- List<String> colNames = Arrays.asList("a", "b");
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
" PARTITIONED BY(ds string)" +
" CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
" STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
executeStatementOnDriver("insert into " + tblName + " partition (ds) values (1, 'fred', " +
- "'today'), (2, 'wilma', 'yesterday')", driver);
+ "'today'), (2, 'wilma', 'yesterday')", driver);
Initiator initiator = new Initiator();
- initiator.setThreadId((int)initiator.getId());
+ initiator.setThreadId((int) initiator.getId());
conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 0);
initiator.setConf(conf);
AtomicBoolean stop = new AtomicBoolean();
@@ -543,19 +587,18 @@ public class TestCompactor {
@Test
public void dynamicPartitioningUpdate() throws Exception {
String tblName = "udpct";
- List<String> colNames = Arrays.asList("a", "b");
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
" PARTITIONED BY(ds string)" +
" CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
" STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
executeStatementOnDriver("insert into " + tblName + " partition (ds) values (1, 'fred', " +
- "'today'), (2, 'wilma', 'yesterday')", driver);
+ "'today'), (2, 'wilma', 'yesterday')", driver);
executeStatementOnDriver("update " + tblName + " set b = 'barney'", driver);
Initiator initiator = new Initiator();
- initiator.setThreadId((int)initiator.getId());
+ initiator.setThreadId((int) initiator.getId());
// Set to 1 so insert doesn't set it off but update does
conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 1);
initiator.setConf(conf);
@@ -583,21 +626,20 @@ public class TestCompactor {
@Test
public void dynamicPartitioningDelete() throws Exception {
String tblName = "ddpct";
- List<String> colNames = Arrays.asList("a", "b");
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
" PARTITIONED BY(ds string)" +
" CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
" STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
executeStatementOnDriver("insert into " + tblName + " partition (ds) values (1, 'fred', " +
- "'today'), (2, 'wilma', 'yesterday')", driver);
+ "'today'), (2, 'wilma', 'yesterday')", driver);
executeStatementOnDriver("update " + tblName + " set b = 'fred' where a = 1", driver);
executeStatementOnDriver("delete from " + tblName + " where b = 'fred'", driver);
Initiator initiator = new Initiator();
- initiator.setThreadId((int)initiator.getId());
+ initiator.setThreadId((int) initiator.getId());
// Set to 2 so insert and update don't set it off but delete does
conf.setIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_DELTA_NUM_THRESHOLD, 2);
initiator.setConf(conf);
@@ -625,25 +667,22 @@ public class TestCompactor {
public void minorCompactWhileStreaming() throws Exception {
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
+ StreamingConnection connection = null;
try {
// Write a couple of batches
for (int i = 0; i < 2; i++) {
- writeBatch(connection, writer, false);
+ writeBatch(dbName, tblName, false);
}
// Start a third batch, but don't close it.
- writeBatch(connection, writer, true);
+ connection = writeBatch(dbName, tblName, true);
// Now, compact
TxnStore txnHandler = TxnUtils.getTxnStore(conf);
@@ -661,7 +700,7 @@ public class TestCompactor {
Table table = msClient.getTable(dbName, tblName);
FileSystem fs = FileSystem.get(conf);
FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
String[] names = new String[stat.length];
Path resultFile = null;
for (int i = 0; i < names.length; i++) {
@@ -672,15 +711,17 @@ public class TestCompactor {
}
Arrays.sort(names);
String[] expected = new String[]{"delta_0000001_0000002",
- "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"};
+ "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"};
if (!Arrays.deepEquals(expected, names)) {
Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names));
}
checkExpectedTxnsPresent(null, new Path[]{resultFile}, columnNamesProperty, columnTypesProperty,
- 0, 1L, 4L, 1);
+ 0, 1L, 4L, 1);
} finally {
- connection.close();
+ if (connection != null) {
+ connection.close();
+ }
}
}
@@ -688,26 +729,23 @@ public class TestCompactor {
public void majorCompactWhileStreaming() throws Exception {
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true') ", driver);
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true') ", driver);
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
+ StreamingConnection connection = null;
try {
// Write a couple of batches
for (int i = 0; i < 2; i++) {
- writeBatch(connection, writer, false);
+ writeBatch(dbName, tblName, false);
}
// Start a third batch, but don't close it. this delta will be ignored by compaction since
// it has an open txn in it
- writeBatch(connection, writer, true);
+ connection = writeBatch(dbName, tblName, true);
runMajorCompaction(dbName, tblName);
@@ -716,7 +754,7 @@ public class TestCompactor {
Table table = msClient.getTable(dbName, tblName);
FileSystem fs = FileSystem.get(conf);
FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
if (1 != stat.length) {
Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat));
}
@@ -724,127 +762,187 @@ public class TestCompactor {
Assert.assertEquals(name, "base_0000004");
checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1);
} finally {
- connection.close();
+ if (connection != null) {
+ connection.close();
+ }
}
}
@Test
public void minorCompactAfterAbort() throws Exception {
- String agentInfo = "UT_" + Thread.currentThread().getName();
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
- try {
- // Write a couple of batches
- for (int i = 0; i < 2; i++) {
- 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();
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
- // Now, compact
- TxnStore txnHandler = TxnUtils.getTxnStore(conf);
- txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR));
- Worker t = new Worker();
- t.setThreadId((int) t.getId());
- t.setConf(conf);
- AtomicBoolean stop = new AtomicBoolean(true);
- AtomicBoolean looped = new AtomicBoolean();
- t.init(stop, looped);
- t.run();
+ if (newStreamingAPI) {
+ StreamingConnection connection = null;
+ try {
+ // Write a couple of batches
+ for (int i = 0; i < 2; i++) {
+ connection = writeBatch(dbName, tblName, false);
+ assertNull(connection);
+ }
- // Find the location of the table
- IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
- Table table = msClient.getTable(dbName, tblName);
- FileSystem fs = FileSystem.get(conf);
- FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
- String[] names = new String[stat.length];
- Path resultDelta = null;
- for (int i = 0; i < names.length; i++) {
- names[i] = stat[i].getPath().getName();
- if (names[i].equals("delta_0000001_0000004")) {
- resultDelta = stat[i].getPath();
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ StreamingConnection connection2 = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(2)
+ .connect();
+ // Start a third batch, abortTransaction everything, don't properly close it
+ connection2.beginTransaction();
+ connection2.abortTransaction();
+ connection2.close();
+ } finally {
+ if (connection != null) {
+ connection.close();
}
}
- Arrays.sort(names);
- String[] expected = new String[]{"delta_0000001_0000002",
- "delta_0000001_0000004", "delta_0000003_0000004"};
- if (!Arrays.deepEquals(expected, names)) {
- Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names));
+ } 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++) {
+ 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();
}
- checkExpectedTxnsPresent(null, new Path[]{resultDelta}, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1);
- } finally {
- connection.close();
}
+ // Now, compact
+ TxnStore txnHandler = TxnUtils.getTxnStore(conf);
+ txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR));
+ Worker t = new Worker();
+ t.setThreadId((int) t.getId());
+ t.setConf(conf);
+ AtomicBoolean stop = new AtomicBoolean(true);
+ AtomicBoolean looped = new AtomicBoolean();
+ t.init(stop, looped);
+ t.run();
+
+ // Find the location of the table
+ IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
+ Table table = msClient.getTable(dbName, tblName);
+ FileSystem fs = FileSystem.get(conf);
+ FileStatus[] stat =
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
+ String[] names = new String[stat.length];
+ Path resultDelta = null;
+ for (int i = 0; i < names.length; i++) {
+ names[i] = stat[i].getPath().getName();
+ if (names[i].equals("delta_0000001_0000004")) {
+ resultDelta = stat[i].getPath();
+ }
+ }
+ Arrays.sort(names);
+ String[] expected = new String[]{"delta_0000001_0000002",
+ "delta_0000001_0000004", "delta_0000003_0000004"};
+ if (!Arrays.deepEquals(expected, names)) {
+ Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names));
+ }
+ checkExpectedTxnsPresent(null, new Path[]{resultDelta}, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1);
}
@Test
public void majorCompactAfterAbort() throws Exception {
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
- try {
- // Write a couple of batches
- for (int i = 0; i < 2; i++) {
- writeBatch(connection, writer, false);
- }
-
- // Start a third batch, but don't close it.
- TransactionBatch txnBatch = connection.fetchTransactionBatch(2, writer);
- txnBatch.beginNextTransaction();
- txnBatch.abort();
- txnBatch.beginNextTransaction();
- txnBatch.abort();
-
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
- runMajorCompaction(dbName, tblName);
+ if (newStreamingAPI) {
+ StreamingConnection connection = null;
+ try {
+ // Write a couple of batches
+ for (int i = 0; i < 2; i++) {
+ connection = writeBatch(dbName, tblName, false);
+ assertNull(connection);
+ }
- // Find the location of the table
- IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
- Table table = msClient.getTable(dbName, tblName);
- FileSystem fs = FileSystem.get(conf);
- FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
- if (1 != stat.length) {
- Assert.fail("majorCompactAfterAbort FileStatus[] stat " + Arrays.toString(stat));
- }
- if (1 != stat.length) {
- Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat));
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ StreamingConnection connection2 = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(2)
+ .connect();
+ // Start a third batch, but don't close it.
+ connection2.beginTransaction();
+ connection2.abortTransaction();
+ connection2.close();
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
}
- String name = stat[0].getPath().getName();
- if (!name.equals("base_0000004")) {
- Assert.fail("majorCompactAfterAbort name " + name + " not equals to base_0000004");
+ } 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++) {
+ 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();
}
- checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1);
- } finally {
- connection.close();
}
+ runMajorCompaction(dbName, tblName);
+
+ // Find the location of the table
+ IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
+ Table table = msClient.getTable(dbName, tblName);
+ FileSystem fs = FileSystem.get(conf);
+ FileStatus[] stat =
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
+ if (1 != stat.length) {
+ Assert.fail("majorCompactAfterAbort FileStatus[] stat " + Arrays.toString(stat));
+ }
+ if (1 != stat.length) {
+ Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat));
+ }
+ String name = stat[0].getPath().getName();
+ if (!name.equals("base_0000004")) {
+ Assert.fail("majorCompactAfterAbort name " + name + " not equals to base_0000004");
+ }
+ checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 0, 1L, 4L, 1);
}
@@ -1073,68 +1171,79 @@ public class TestCompactor {
public void majorCompactWhileStreamingForSplitUpdate() throws Exception {
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true', "
- + "'transactional_properties'='default') ", driver); // this turns on split-update U=D+I
+ " CLUSTERED BY(a) INTO 2 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true', "
+ + "'transactional_properties'='default') ", driver); // this turns on split-update U=D+I
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
- try {
+ 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++) {
- writeBatch(connection, writer, false);
+ writeBatch(dbName, tblName, false);
}
// Start a third batch, but don't close it.
- writeBatch(connection, writer, true);
+ connection1 = writeBatch(dbName, tblName, 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++) {
+ writeBatch(connection2, writer, false);
+ }
- runMajorCompaction(dbName, tblName);
+ // Start a third batch, but don't close it.
+ writeBatch(connection2, writer, true);
+ }
+ runMajorCompaction(dbName, tblName);
- // Find the location of the table
- IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
- Table table = msClient.getTable(dbName, tblName);
- FileSystem fs = FileSystem.get(conf);
- FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
- if (1 != stat.length) {
- Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat));
- }
- String name = stat[0].getPath().getName();
- Assert.assertEquals(name, "base_0000004");
- checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 1, 1L, 4L, 2);
- } finally {
- connection.close();
+ // Find the location of the table
+ IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
+ Table table = msClient.getTable(dbName, tblName);
+ FileSystem fs = FileSystem.get(conf);
+ FileStatus[] stat =
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.baseFileFilter);
+ if (1 != stat.length) {
+ Assert.fail("Expecting 1 file \"base_0000004\" and found " + stat.length + " files " + Arrays.toString(stat));
+ }
+ String name = stat[0].getPath().getName();
+ Assert.assertEquals(name, "base_0000004");
+ checkExpectedTxnsPresent(stat[0].getPath(), null, columnNamesProperty, columnTypesProperty, 1, 1L, 4L, 2);
+ if (connection1 != null) {
+ connection1.close();
+ }
+ if (connection2 != null) {
+ connection2.close();
}
}
@Test
public void testMinorCompactionForSplitUpdateWithInsertsAndDeletes() throws Exception {
- String agentInfo = "UT_" + Thread.currentThread().getName();
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
- + "'transactional_properties'='default')", driver);
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
+ + "'transactional_properties'='default')", driver);
// Insert some data -> this will generate only insert deltas and no delete deltas: delta_3_3
- executeStatementOnDriver("INSERT INTO " + tblName +"(a,b) VALUES(1, 'foo')", driver);
+ executeStatementOnDriver("INSERT INTO " + tblName + "(a,b) VALUES(1, 'foo')", driver);
// Insert some data -> this will again generate only insert deltas and no delete deltas: delta_4_4
- executeStatementOnDriver("INSERT INTO " + tblName +"(a,b) VALUES(2, 'bar')", driver);
+ executeStatementOnDriver("INSERT INTO " + tblName + "(a,b) VALUES(2, 'bar')", driver);
// Delete some data -> this will generate only delete deltas and no insert deltas: delete_delta_5_5
- executeStatementOnDriver("DELETE FROM " + tblName +" WHERE a = 2", driver);
+ executeStatementOnDriver("DELETE FROM " + tblName + " WHERE a = 2", driver);
// Now, compact -> Compaction produces a single range for both delta and delete delta
// That is, both delta and delete_deltas would be compacted into delta_3_5 and delete_delta_3_5
@@ -1156,7 +1265,7 @@ public class TestCompactor {
// Verify that we have got correct set of deltas.
FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
String[] deltas = new String[stat.length];
Path minorCompactedDelta = null;
for (int i = 0; i < deltas.length; i++) {
@@ -1166,16 +1275,17 @@ public class TestCompactor {
}
}
Arrays.sort(deltas);
- String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000003", "delta_0000002_0000002_0000"};
+ String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000003",
+ "delta_0000002_0000002_0000"};
if (!Arrays.deepEquals(expectedDeltas, deltas)) {
Assert.fail("Expected: " + Arrays.toString(expectedDeltas) + ", found: " + Arrays.toString(deltas));
}
checkExpectedTxnsPresent(null, new Path[]{minorCompactedDelta}, columnNamesProperty, columnTypesProperty,
- 0, 1L, 2L, 1);
+ 0, 1L, 2L, 1);
// Verify that we have got correct set of delete_deltas.
FileStatus[] deleteDeltaStat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
String[] deleteDeltas = new String[deleteDeltaStat.length];
Path minorCompactedDeleteDelta = null;
for (int i = 0; i < deleteDeltas.length; i++) {
@@ -1190,28 +1300,26 @@ public class TestCompactor {
Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas));
}
checkExpectedTxnsPresent(null, new Path[]{minorCompactedDeleteDelta}, columnNamesProperty, columnTypesProperty,
- 0, 2L, 2L, 1);
+ 0, 2L, 2L, 1);
}
@Test
public void testMinorCompactionForSplitUpdateWithOnlyInserts() throws Exception {
- String agentInfo = "UT_" + Thread.currentThread().getName();
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
- + "'transactional_properties'='default')", driver);
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
+ + "'transactional_properties'='default')", driver);
// Insert some data -> this will generate only insert deltas and no delete deltas: delta_1_1
- executeStatementOnDriver("INSERT INTO " + tblName +"(a,b) VALUES(1, 'foo')", driver);
+ executeStatementOnDriver("INSERT INTO " + tblName + "(a,b) VALUES(1, 'foo')", driver);
// Insert some data -> this will again generate only insert deltas and no delete deltas: delta_2_2
- executeStatementOnDriver("INSERT INTO " + tblName +"(a,b) VALUES(2, 'bar')", driver);
+ executeStatementOnDriver("INSERT INTO " + tblName + "(a,b) VALUES(2, 'bar')", driver);
// Now, compact
// One important thing to note in this test is that minor compaction always produces
@@ -1235,7 +1343,7 @@ public class TestCompactor {
// Verify that we have got correct set of deltas.
FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
String[] deltas = new String[stat.length];
Path minorCompactedDelta = null;
for (int i = 0; i < deltas.length; i++) {
@@ -1245,16 +1353,17 @@ public class TestCompactor {
}
}
Arrays.sort(deltas);
- String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000002", "delta_0000002_0000002_0000"};
+ String[] expectedDeltas = new String[]{"delta_0000001_0000001_0000", "delta_0000001_0000002",
+ "delta_0000002_0000002_0000"};
if (!Arrays.deepEquals(expectedDeltas, deltas)) {
Assert.fail("Expected: " + Arrays.toString(expectedDeltas) + ", found: " + Arrays.toString(deltas));
}
checkExpectedTxnsPresent(null, new Path[]{minorCompactedDelta}, columnNamesProperty, columnTypesProperty,
- 0, 1L, 2L, 1);
+ 0, 1L, 2L, 1);
// Verify that we have got correct set of delete_deltas.
FileStatus[] deleteDeltaStat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
String[] deleteDeltas = new String[deleteDeltaStat.length];
Path minorCompactedDeleteDelta = null;
for (int i = 0; i < deleteDeltas.length; i++) {
@@ -1269,89 +1378,105 @@ public class TestCompactor {
Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas));
}
// There should be no rows in the delete_delta because there have been no delete events.
- checkExpectedTxnsPresent(null, new Path[]{minorCompactedDeleteDelta}, columnNamesProperty, columnTypesProperty, 0, 0L, 0L, 1);
+ checkExpectedTxnsPresent(null, new Path[]{minorCompactedDeleteDelta}, columnNamesProperty, columnTypesProperty, 0,
+ 0L, 0L, 1);
}
@Test
public void minorCompactWhileStreamingWithSplitUpdate() throws Exception {
String dbName = "default";
String tblName = "cws";
- List<String> colNames = Arrays.asList("a", "b");
String columnNamesProperty = "a,b";
String columnTypesProperty = "int:string";
executeStatementOnDriver("drop table if exists " + tblName, driver);
executeStatementOnDriver("CREATE TABLE " + tblName + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
- " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
- + "'transactional_properties'='default')", driver);
+ " CLUSTERED BY(a) INTO 1 BUCKETS" + //currently ACID requires table to be bucketed
+ " STORED AS ORC TBLPROPERTIES ('transactional'='true',"
+ + "'transactional_properties'='default')", driver);
+
+ StreamingConnection connection1 = null;
+ org.apache.hive.hcatalog.streaming.StreamingConnection connection2 = null;
+ if (newStreamingAPI) {
- HiveEndPoint endPt = new HiveEndPoint(null, dbName, tblName, null);
- DelimitedInputWriter writer = new DelimitedInputWriter(new String[] {"a","b"},",", endPt);
- StreamingConnection connection = endPt.newConnection(false, "UT_" + Thread.currentThread().getName());
- try {
// Write a couple of batches
for (int i = 0; i < 2; i++) {
- writeBatch(connection, writer, false);
+ writeBatch(dbName, tblName, false);
}
// Start a third batch, but don't close it.
- writeBatch(connection, writer, true);
+ connection1 = writeBatch(dbName, tblName, 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++) {
+ writeBatch(connection2, writer, false);
+ }
- // Now, compact
- TxnStore txnHandler = TxnUtils.getTxnStore(conf);
- txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR));
- Worker t = new Worker();
- t.setThreadId((int) t.getId());
- t.setConf(conf);
- AtomicBoolean stop = new AtomicBoolean(true);
- AtomicBoolean looped = new AtomicBoolean();
- t.init(stop, looped);
- t.run();
+ // Start a third batch, but don't close it.
+ writeBatch(connection2, writer, true);
+ }
+ // Now, compact
+ TxnStore txnHandler = TxnUtils.getTxnStore(conf);
+ txnHandler.compact(new CompactionRequest(dbName, tblName, CompactionType.MINOR));
+ Worker t = new Worker();
+ t.setThreadId((int) t.getId());
+ t.setConf(conf);
+ AtomicBoolean stop = new AtomicBoolean(true);
+ AtomicBoolean looped = new AtomicBoolean();
+ t.init(stop, looped);
+ t.run();
- // Find the location of the table
- IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
- Table table = msClient.getTable(dbName, tblName);
- FileSystem fs = FileSystem.get(conf);
- FileStatus[] stat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
- String[] names = new String[stat.length];
- Path resultFile = null;
- for (int i = 0; i < names.length; i++) {
- names[i] = stat[i].getPath().getName();
- if (names[i].equals("delta_0000001_0000004")) {
- resultFile = stat[i].getPath();
- }
- }
- Arrays.sort(names);
- String[] expected = new String[]{"delta_0000001_0000002",
- "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"};
- if (!Arrays.deepEquals(expected, names)) {
- Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names));
- }
- checkExpectedTxnsPresent(null, new Path[]{resultFile}, columnNamesProperty, columnTypesProperty,
- 0, 1L, 4L, 1);
-
- // Verify that we have got correct set of delete_deltas also
- FileStatus[] deleteDeltaStat =
- fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
- String[] deleteDeltas = new String[deleteDeltaStat.length];
- Path minorCompactedDeleteDelta = null;
- for (int i = 0; i < deleteDeltas.length; i++) {
- deleteDeltas[i] = deleteDeltaStat[i].getPath().getName();
- if (deleteDeltas[i].equals("delete_delta_0000001_0000004")) {
- minorCompactedDeleteDelta = deleteDeltaStat[i].getPath();
- }
+ // Find the location of the table
+ IMetaStoreClient msClient = new HiveMetaStoreClient(conf);
+ Table table = msClient.getTable(dbName, tblName);
+ FileSystem fs = FileSystem.get(conf);
+ FileStatus[] stat =
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deltaFileFilter);
+ String[] names = new String[stat.length];
+ Path resultFile = null;
+ for (int i = 0; i < names.length; i++) {
+ names[i] = stat[i].getPath().getName();
+ if (names[i].equals("delta_0000001_0000004")) {
+ resultFile = stat[i].getPath();
}
- Arrays.sort(deleteDeltas);
- String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000004"};
- if (!Arrays.deepEquals(expectedDeleteDeltas, deleteDeltas)) {
- Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas));
+ }
+ Arrays.sort(names);
+ String[] expected = new String[]{"delta_0000001_0000002",
+ "delta_0000001_0000004", "delta_0000003_0000004", "delta_0000005_0000006"};
+ if (!Arrays.deepEquals(expected, names)) {
+ Assert.fail("Expected: " + Arrays.toString(expected) + ", found: " + Arrays.toString(names));
+ }
+ checkExpectedTxnsPresent(null, new Path[]{resultFile}, columnNamesProperty, columnTypesProperty,
+ 0, 1L, 4L, 1);
+
+ // Verify that we have got correct set of delete_deltas also
+ FileStatus[] deleteDeltaStat =
+ fs.listStatus(new Path(table.getSd().getLocation()), AcidUtils.deleteEventDeltaDirFilter);
+ String[] deleteDeltas = new String[deleteDeltaStat.length];
+ Path minorCompactedDeleteDelta = null;
+ for (int i = 0; i < deleteDeltas.length; i++) {
+ deleteDeltas[i] = deleteDeltaStat[i].getPath().getName();
+ if (deleteDeltas[i].equals("delete_delta_0000001_0000004")) {
+ minorCompactedDeleteDelta = deleteDeltaStat[i].getPath();
}
- // There should be no rows in the delete_delta because there have been no delete events.
- checkExpectedTxnsPresent(null, new Path[]{minorCompactedDeleteDelta}, columnNamesProperty, columnTypesProperty, 0, 0L, 0L, 1);
+ }
+ Arrays.sort(deleteDeltas);
+ String[] expectedDeleteDeltas = new String[]{"delete_delta_0000001_0000004"};
+ if (!Arrays.deepEquals(expectedDeleteDeltas, deleteDeltas)) {
+ Assert.fail("Expected: " + Arrays.toString(expectedDeleteDeltas) + ", found: " + Arrays.toString(deleteDeltas));
+ }
+ // There should be no rows in the delete_delta because there have been no delete events.
+ checkExpectedTxnsPresent(null, new Path[]{minorCompactedDeleteDelta}, columnNamesProperty, columnTypesProperty, 0,
+ 0L, 0L, 1);
- } finally {
- connection.close();
+ if (connection1 != null) {
+ connection1.close();
+ }
+ if (connection2 != null) {
+ connection2.close();
}
}
@@ -1366,15 +1491,15 @@ public class TestCompactor {
executeStatementOnDriver("drop table if exists " + tblName1, driver);
executeStatementOnDriver("drop table if exists " + tblName2, driver);
executeStatementOnDriver("CREATE TABLE " + tblName1 + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC" +
- " TBLPROPERTIES ('transactional'='true', 'orc.compress.size'='2700')", driver);
+ " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC" +
+ " TBLPROPERTIES ('transactional'='true', 'orc.compress.size'='2700')", driver);
executeStatementOnDriver("CREATE TABLE " + tblName2 + "(a INT, b STRING) " +
- " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES (" +
- "'transactional'='true'," +
- "'compactor.mapreduce.map.memory.mb'='2048'," + // 2048 MB memory for compaction map job
- "'compactorthreshold.hive.compactor.delta.num.threshold'='4'," + // minor compaction if more than 4 delta dirs
- "'compactorthreshold.hive.compactor.delta.pct.threshold'='0.49'" + // major compaction if more than 49%
- ")", driver);
+ " CLUSTERED BY(a) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES (" +
+ "'transactional'='true'," +
+ "'compactor.mapreduce.map.memory.mb'='2048'," + // 2048 MB memory for compaction map job
+ "'compactorthreshold.hive.compactor.delta.num.threshold'='4'," + // minor compaction if more than 4 delta dirs
+ "'compactorthreshold.hive.compactor.delta.pct.threshold'='0.49'" + // major compaction if more than 49%
+ ")", driver);
// Insert 5 rows to both tables
executeStatementOnDriver("insert into " + tblName1 + " values (1, 'a')", driver);
@@ -1397,7 +1522,8 @@ public class TestCompactor {
Assert.assertEquals(1, rsp.getCompacts().size());
Assert.assertEquals(TxnStore.INITIATED_RESPONSE, rsp.getCompacts().get(0).getState());
Assert.assertEquals("ttp2", rsp.getCompacts().get(0).getTablename());
- Assert.assertEquals(CompactionType.MAJOR, rsp.getCompacts().get(0).getType()); // type is MAJOR since there's no base yet
+ Assert.assertEquals(CompactionType.MAJOR,
+ rsp.getCompacts().get(0).getType()); // type is MAJOR since there's no base yet
// Finish the scheduled compaction for ttp2, and manually compact ttp1, to make them comparable again
executeStatementOnDriver("alter table " + tblName1 + " compact 'major'", driver);
@@ -1441,15 +1567,15 @@ public class TestCompactor {
* It should be the default.
*/
List<String> rs = execSelectAndDumpData("select distinct INPUT__FILE__NAME from "
- + tblName1, driver, "Find Orc File bufer default");
+ + tblName1, driver, "Find Orc File bufer default");
Assert.assertTrue("empty rs?", rs != null && rs.size() > 0);
Path p = new Path(rs.get(0));
Reader orcReader = OrcFile.createReader(p.getFileSystem(conf), p);
Assert.assertEquals("Expected default compression size",
- 2700, orcReader.getCompressionSize());
+ 2700, orcReader.getCompressionSize());
//make sure 2700 is not the default so that we are testing if tblproperties indeed propagate
Assert.assertNotEquals("Unexpected default compression size", 2700,
- OrcConf.BUFFER_SIZE.getDefaultValue());
+ OrcConf.BUFFER_SIZE.getDefaultValue());
// Insert one more row - this should trigger hive.compactor.delta.pct.threshold to be reached for ttp2
@@ -1476,9 +1602,9 @@ public class TestCompactor {
// Now test tblproperties specified on ALTER TABLE .. COMPACT .. statement
executeStatementOnDriver("insert into " + tblName2 + " values (7, 'g')", driver);
executeStatementOnDriver("alter table " + tblName2 + " compact 'major'" +
- " with overwrite tblproperties (" +
- "'compactor.mapreduce.map.memory.mb'='3072'," +
- "'tblprops.orc.compress.size'='3141')", driver);
+ " with overwrite tblproperties (" +
+ "'compactor.mapreduce.map.memory.mb'='3072'," +
+ "'tblprops.orc.compress.size'='3141')", driver);
rsp = txnHandler.showCompact(new ShowCompactRequest());
Assert.assertEquals(4, rsp.getCompacts().size());
@@ -1499,16 +1625,16 @@ public class TestCompactor {
Assert.assertEquals(3072, job.getMemoryForMapTask());
Assert.assertTrue(job.get("hive.compactor.table.props").contains("orc.compress.size4:3141"));
/*createReader(FileSystem fs, Path path) throws IOException {
- */
+ */
//we just ran Major compaction so we should have a base_x in tblName2 that has the new files
// Get the name of a file and look at its properties to see if orc.compress.size was respected.
rs = execSelectAndDumpData("select distinct INPUT__FILE__NAME from " + tblName2,
- driver, "Find Compacted Orc File");
+ driver, "Find Compacted Orc File");
Assert.assertTrue("empty rs?", rs != null && rs.size() > 0);
p = new Path(rs.get(0));
orcReader = OrcFile.createReader(p.getFileSystem(conf), p);
Assert.assertEquals("File written with wrong buffer size",
- 3141, orcReader.getCompressionSize());
+ 3141, orcReader.getCompressionSize());
}
@Test
@@ -1529,30 +1655,58 @@ public class TestCompactor {
Assert.assertEquals("The hash codes must be equal", compactionInfo.hashCode(), compactionInfo1.hashCode());
}
- private void writeBatch(StreamingConnection connection, DelimitedInputWriter writer,
- boolean closeEarly)
- throws InterruptedException, StreamingException {
+ private 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();
}
}
+ private StreamingConnection writeBatch(String dbName, String tblName, boolean closeEarly) throws StreamingException {
+ StrictDelimitedInputWriter writer = StrictDelimitedInputWriter.newBuilder()
+ .withFieldDelimiter(',')
+ .build();
+ StreamingConnection connection = HiveStreamingConnection.newBuilder()
+ .withDatabase(dbName)
+ .withTable(tblName)
+ .withAgentInfo("UT_" + Thread.currentThread().getName())
+ .withHiveConf(conf)
+ .withRecordWriter(writer)
+ .withTransactionBatchSize(2)
+ .connect();
+ connection.beginTransaction();
+ connection.write("50,Kiev".getBytes());
+ connection.write("51,St. Petersburg".getBytes());
+ connection.write("44,Boston".getBytes());
+ connection.commitTransaction();
+
+ if (!closeEarly) {
+ connection.beginTransaction();
+ connection.write("52,Tel Aviv".getBytes());
+ connection.write("53,Atlantis".getBytes());
+ connection.write("53,Boston".getBytes());
+ connection.commitTransaction();
+ connection.close();
+ return null;
+ }
+ return connection;
+ }
+
private void checkExpectedTxnsPresent(Path base, Path[] deltas, String columnNamesProperty,
- String columnTypesProperty, int bucket, long min, long max, int numBuckets)
- throws IOException {
+ String columnTypesProperty, int bucket, long min, long max, int numBuckets)
+ throws IOException {
ValidWriteIdList writeIdList = new ValidWriteIdList() {
@Override
public String getTableName() {
@@ -1586,13 +1740,14 @@ public class TestCompactor {
@Override
public long getHighWatermark() {
- return Long.MAX_VALUE;
+ return Long.MAX_VALUE;
}
@Override
public long[] getInvalidWriteIds() {
return new long[0];
}
+
@Override
public boolean isValidBase(long writeid) {
return true;
@@ -1617,7 +1772,7 @@ public class TestCompactor {
conf.set(hive_metastoreConstants.BUCKET_COUNT, Integer.toString(numBuckets));
HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_TRANSACTIONAL_TABLE_SCAN, true);
AcidInputFormat.RawReader<OrcStruct> reader =
- aif.getRawReader(conf, true, bucket, writeIdList, base, deltas);
+ aif.getRawReader(conf, true, bucket, writeIdList, base, deltas);
RecordIdentifier identifier = reader.createKey();
OrcStruct value = reader.createValue();
long currentTxn = min;
@@ -1639,25 +1794,27 @@ public class TestCompactor {
* convenience method to execute a select stmt and dump results to log file
*/
private static List<String> execSelectAndDumpData(String selectStmt, IDriver driver, String msg)
- throws Exception {
+ throws Exception {
executeStatementOnDriver(selectStmt, driver);
ArrayList<String> valuesReadFromHiveDriver = new ArrayList<String>();
driver.getResults(valuesReadFromHiveDriver);
int rowIdx = 0;
LOG.debug(msg);
- for(String row : valuesReadFromHiveDriver) {
+ for (String row : valuesReadFromHiveDriver) {
LOG.debug(" rowIdx=" + rowIdx++ + ":" + row);
}
return valuesReadFromHiveDriver;
}
+
/**
* Execute Hive CLI statement
+ *
* @param cmd arbitrary statement to execute
*/
static void executeStatementOnDriver(String cmd, IDriver driver) throws Exception {
LOG.debug("Executing: " + cmd);
CommandProcessorResponse cpr = driver.run(cmd);
- if(cpr.getResponseCode() != 0) {
+ if (cpr.getResponseCode() != 0) {
throw new IOException("Failed to execute \"" + cmd + "\". Driver returned: " + cpr);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveClientCache.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveClientCache.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveClientCache.java
new file mode 100644
index 0000000..6c33f63
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveClientCache.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.hive.metastore;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.shims.Utils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.common.util.ShutdownHookManager;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A thread safe time expired cache for HiveMetaStoreClient
+ */
+class HiveClientCache {
+ public final static int DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS = 2 * 60;
+ public final static int DEFAULT_HIVE_CACHE_INITIAL_CAPACITY = 50;
+ public final static int DEFAULT_HIVE_CACHE_MAX_CAPACITY = 50;
+ public final static boolean DEFAULT_HIVE_CLIENT_CACHE_STATS_ENABLED = false;
+
+ private final Cache<HiveClientCacheKey, ICacheableMetaStoreClient> hiveCache;
+ private static final Logger LOG = LoggerFactory.getLogger(HiveClientCache.class);
+ private final int timeout;
+ // This lock is used to make sure removalListener won't close a client that is being contemplated for returning by get()
+ private final Object CACHE_TEARDOWN_LOCK = new Object();
+
+ private static final AtomicInteger nextId = new AtomicInteger(0);
+
+ private final ScheduledFuture<?> cleanupHandle; // used to cleanup cache
+
+ private boolean enableStats;
+
+ // Since HiveMetaStoreClient is not threadsafe, hive clients are not shared across threads.
+ // Thread local variable containing each thread's unique ID, is used as one of the keys for the cache
+ // causing each thread to get a different client even if the conf is same.
+ private static final ThreadLocal<Integer> threadId =
+ new ThreadLocal<Integer>() {
+ @Override
+ protected Integer initialValue() {
+ return nextId.getAndIncrement();
+ }
+ };
+
+ private int getThreadId() {
+ return threadId.get();
+ }
+
+ public static IMetaStoreClient getNonCachedHiveMetastoreClient(HiveConf hiveConf) throws MetaException {
+ return RetryingMetaStoreClient.getProxy(hiveConf, true);
+ }
+
+ public HiveClientCache(HiveConf hiveConf) {
+ this((int) HiveConf.getTimeVar(hiveConf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_EXPIRY_TIME, TimeUnit.SECONDS),
+ HiveConf.getIntVar(hiveConf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_INITIAL_CAPACITY),
+ HiveConf.getIntVar(hiveConf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_MAX_CAPACITY),
+ HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_STATS_ENABLED));
+ }
+
+ /**
+ * @deprecated This constructor will be made private or removed as more configuration properties are required.
+ */
+ @Deprecated
+ public HiveClientCache(final int timeout) {
+ this(timeout, DEFAULT_HIVE_CACHE_INITIAL_CAPACITY, DEFAULT_HIVE_CACHE_MAX_CAPACITY, DEFAULT_HIVE_CLIENT_CACHE_STATS_ENABLED);
+ }
+
+ /**
+ * @param timeout the length of time in seconds after a client is created that it should be automatically removed
+ */
+ private HiveClientCache(final int timeout, final int initialCapacity, final int maxCapacity, final boolean enableStats) {
+ this.timeout = timeout;
+ this.enableStats = enableStats;
+
+ LOG.info("Initializing cache: eviction-timeout=" + timeout + " initial-capacity=" + initialCapacity + " maximum-capacity=" + maxCapacity);
+
+ CacheBuilder builder = CacheBuilder.newBuilder()
+ .initialCapacity(initialCapacity)
+ .maximumSize(maxCapacity)
+ .expireAfterAccess(timeout, TimeUnit.SECONDS)
+ .removalListener(createRemovalListener());
+
+ /*
+ * Guava versions <12.0 have stats collection enabled by default and do not expose a recordStats method.
+ * Check for newer versions of the library and ensure that stats collection is enabled by default.
+ */
+ try {
+ java.lang.reflect.Method m = builder.getClass().getMethod("recordStats", null);
+ m.invoke(builder, null);
+ } catch (NoSuchMethodException e) {
+ LOG.debug("Using a version of guava <12.0. Stats collection is enabled by default.");
+ } catch (Exception e) {
+ LOG.warn("Unable to invoke recordStats method.", e);
+ }
+
+ this.hiveCache = builder.build();
+
+ /*
+ * We need to use a cleanup interval, which is how often the cleanup thread will kick in
+ * and go do a check to see if any of the connections can be expired. We don't want to
+ * do this too often, because it'd be like having a mini-GC going off every so often,
+ * so we limit it to a minimum of DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS. If the client
+ * has explicitly set a larger timeout on the cache, though, we respect that, and use that
+ */
+ long cleanupInterval = timeout > DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS ? timeout : DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS;
+
+ this.cleanupHandle = createCleanupThread(cleanupInterval);
+
+ createShutdownHook();
+ }
+
+ private RemovalListener<HiveClientCacheKey, ICacheableMetaStoreClient> createRemovalListener() {
+ RemovalListener<HiveClientCacheKey, ICacheableMetaStoreClient> listener =
+ new RemovalListener<HiveClientCacheKey, ICacheableMetaStoreClient>() {
+ @Override
+ public void onRemoval(RemovalNotification<HiveClientCacheKey, ICacheableMetaStoreClient> notification) {
+ ICacheableMetaStoreClient hiveMetaStoreClient = notification.getValue();
+ if (hiveMetaStoreClient != null) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Evicting client: " + Integer.toHexString(System.identityHashCode(hiveMetaStoreClient)));
+ }
+
+ // TODO: This global lock may not be necessary as all concurrent methods in ICacheableMetaStoreClient
+ // are synchronized.
+ synchronized (CACHE_TEARDOWN_LOCK) {
+ hiveMetaStoreClient.setExpiredFromCache();
+ hiveMetaStoreClient.tearDownIfUnused();
+ }
+ }
+ }
+ };
+
+ return listener;
+ }
+
+ private ScheduledFuture<?> createCleanupThread(long interval) {
+ // Add a maintenance thread that will attempt to trigger a cache clean continuously
+ Runnable cleanupThread = new Runnable() {
+ @Override
+ public void run() {
+ cleanup();
+ }
+ };
+
+ /**
+ * Create the cleanup handle. In addition to cleaning up every cleanupInterval, we add
+ * a slight offset, so that the very first time it runs, it runs with a slight delay, so
+ * as to catch any other connections that were closed when the first timeout happened.
+ * As a result, the time we can expect an unused connection to be reaped is
+ * 5 seconds after the first timeout, and then after that, it'll check for whether or not
+ * it can be cleaned every max(DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS,timeout) seconds
+ */
+ ThreadFactory daemonThreadFactory = (new ThreadFactoryBuilder()).setDaemon(true)
+ .setNameFormat("HiveClientCache-cleaner-%d")
+ .build();
+
+ return Executors.newScheduledThreadPool(1, daemonThreadFactory)
+ .scheduleWithFixedDelay(cleanupThread, timeout + 5, interval, TimeUnit.SECONDS);
+ }
+
+ private void createShutdownHook() {
+ // Add a shutdown hook for cleanup, if there are elements remaining in the cache which were not cleaned up.
+ // This is the best effort approach. Ignore any error while doing so. Notice that most of the clients
+ // would get cleaned up via either the removalListener or the close() call, only the active clients
+ // that are in the cache or expired but being used in other threads wont get cleaned. The following code will only
+ // clean the active cache ones. The ones expired from cache but being hold by other threads are in the mercy
+ // of finalize() being called.
+ Thread cleanupHiveClientShutdownThread = new Thread() {
+ @Override
+ public void run() {
+ LOG.debug("Cleaning up hive client cache in ShutDown hook");
+ cleanupHandle.cancel(false); // Cancel the maintenance thread.
+ closeAllClientsQuietly();
+ }
+ };
+
+ ShutdownHookManager.addShutdownHook(cleanupHiveClientShutdownThread);
+ }
+
+ /**
+ * Note: This doesn't check if they are being used or not, meant only to be called during shutdown etc.
+ */
+ void closeAllClientsQuietly() {
+ try {
+ ConcurrentMap<HiveClientCacheKey, ICacheableMetaStoreClient> elements = hiveCache.asMap();
+ for (ICacheableMetaStoreClient cacheableHiveMetaStoreClient : elements.values()) {
+ cacheableHiveMetaStoreClient.tearDown();
+ }
+ } catch (Exception e) {
+ LOG.warn("Clean up of hive clients in the cache failed. Ignored", e);
+ }
+
+ if (this.enableStats) {
+ LOG.info("Cache statistics after shutdown: size=" + hiveCache.size() + " " + hiveCache.stats());
+ }
+ }
+
+ public void cleanup() {
+ // TODO: periodically reload a new HiveConf to check if stats reporting is enabled.
+ hiveCache.cleanUp();
+
+ if (enableStats) {
+ LOG.info("Cache statistics after cleanup: size=" + hiveCache.size() + " " + hiveCache.stats());
+ }
+ }
+
+ /**
+ * Returns a cached client if exists or else creates one, caches and returns it. It also checks that the client is
+ * healthy and can be reused
+ * @param hiveConf
+ * @return the hive client
+ * @throws MetaException
+ * @throws IOException
+ * @throws LoginException
+ */
+ public IMetaStoreClient get(final HiveConf hiveConf) throws MetaException, IOException, LoginException {
+ final HiveClientCacheKey cacheKey = HiveClientCacheKey.fromHiveConf(hiveConf, getThreadId());
+ ICacheableMetaStoreClient cacheableHiveMetaStoreClient = null;
+
+ // the hmsc is not shared across threads. So the only way it could get closed while we are doing healthcheck
+ // is if removalListener closes it. The synchronization takes care that removalListener won't do it
+ synchronized (CACHE_TEARDOWN_LOCK) {
+ cacheableHiveMetaStoreClient = getOrCreate(cacheKey);
+ cacheableHiveMetaStoreClient.acquire();
+ }
+ if (!cacheableHiveMetaStoreClient.isOpen()) {
+ synchronized (CACHE_TEARDOWN_LOCK) {
+ hiveCache.invalidate(cacheKey);
+ cacheableHiveMetaStoreClient.close();
+ cacheableHiveMetaStoreClient = getOrCreate(cacheKey);
+ cacheableHiveMetaStoreClient.acquire();
+ }
+ }
+ return cacheableHiveMetaStoreClient;
+ }
+
+ /**
+ * Return from cache if exists else create/cache and return
+ * @param cacheKey
+ * @return
+ * @throws IOException
+ * @throws MetaException
+ * @throws LoginException
+ */
+ private ICacheableMetaStoreClient getOrCreate(final HiveClientCacheKey cacheKey)
+ throws IOException, MetaException, LoginException {
+ try {
+ return hiveCache.get(cacheKey, new Callable<ICacheableMetaStoreClient>() {
+ @Override
+ public ICacheableMetaStoreClient call() throws MetaException {
+ // This is called from HCat, so always allow embedded metastore (as was the default).
+ return
+ (ICacheableMetaStoreClient) RetryingMetaStoreClient.getProxy(cacheKey.getHiveConf(),
+ new Class<?>[]{HiveConf.class, Integer.class, Boolean.class},
+ new Object[]{cacheKey.getHiveConf(), timeout, true},
+ CacheableHiveMetaStoreClient.class.getName());
+ }
+ });
+ } catch (ExecutionException e) {
+ Throwable t = e.getCause();
+ if (t instanceof IOException) {
+ throw (IOException) t;
+ } else if (t instanceof MetaException) {
+ throw (MetaException) t;
+ } else if (t instanceof LoginException) {
+ throw (LoginException) t;
+ } else {
+ throw new IOException("Error creating hiveMetaStoreClient", t);
+ }
+ }
+ }
+
+ /**
+ * A class to wrap HiveConf and expose equality based only on UserGroupInformation and the metaStoreURIs.
+ * This becomes the key for the cache and this way the same HiveMetaStoreClient would be returned if
+ * UserGroupInformation and metaStoreURIs are same. This function can evolve to express
+ * the cases when HiveConf is different but the same hiveMetaStoreClient can be used
+ */
+ static class HiveClientCacheKey {
+ final private String metaStoreURIs;
+ final private UserGroupInformation ugi;
+ final private HiveConf hiveConf;
+ final private int threadId;
+
+ private HiveClientCacheKey(HiveConf hiveConf, final int threadId) throws IOException, LoginException {
+ this.metaStoreURIs = hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS);
+ ugi = Utils.getUGI();
+ this.hiveConf = hiveConf;
+ this.threadId = threadId;
+ }
+
+ public static HiveClientCacheKey fromHiveConf(HiveConf hiveConf, final int threadId) throws IOException, LoginException {
+ return new HiveClientCacheKey(hiveConf, threadId);
+ }
+
+ public HiveConf getHiveConf() {
+ return hiveConf;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ HiveClientCacheKey that = (HiveClientCacheKey) o;
+ return new EqualsBuilder().
+ append(this.metaStoreURIs,
+ that.metaStoreURIs).
+ append(this.ugi, that.ugi).
+ append(this.threadId, that.threadId).isEquals();
+ }
+
+ @Override
+ public int hashCode() {
+ return new HashCodeBuilder().
+ append(metaStoreURIs).
+ append(ugi).
+ append(threadId).toHashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "HiveClientCacheKey: uri=" + this.metaStoreURIs + " ugi=" + this.ugi + " thread=" + this.threadId;
+ }
+ }
+
+ @InterfaceAudience.Private
+ public interface ICacheableMetaStoreClient extends IMetaStoreClient {
+ @NoReconnect
+ void acquire();
+
+ @NoReconnect
+ void setExpiredFromCache();
+
+ @NoReconnect
+ AtomicInteger getUsers();
+
+ @NoReconnect
+ boolean isClosed();
+
+ /**
+ * @deprecated This method is not used internally and should not be visible through HCatClient.create.
+ */
+ @Deprecated
+ @NoReconnect
+ boolean isOpen();
+
+ @NoReconnect
+ void tearDownIfUnused();
+
+ @NoReconnect
+ void tearDown();
+ }
+
+ /**
+ * Add # of current users on HiveMetaStoreClient, so that the client can be cleaned when no one is using it.
+ */
+ static class CacheableHiveMetaStoreClient extends HiveMetaStoreClient implements ICacheableMetaStoreClient {
+
+ private final AtomicInteger users = new AtomicInteger(0);
+ private volatile boolean expiredFromCache = false;
+ private boolean isClosed = false;
+
+ CacheableHiveMetaStoreClient(final HiveConf conf, final Integer timeout, Boolean allowEmbedded)
+ throws MetaException {
+ super(conf, null, allowEmbedded);
+ }
+
+ /**
+ * Increments the user count and optionally renews the expiration time.
+ * <code>renew</code> should correspond with the expiration policy of the cache.
+ * When the policy is <code>expireAfterAccess</code>, the expiration time should be extended.
+ * When the policy is <code>expireAfterWrite</code>, the expiration time should not be extended.
+ * A mismatch with the policy will lead to closing the connection unnecessarily after the initial
+ * expiration time is generated.
+ */
+ public synchronized void acquire() {
+ users.incrementAndGet();
+ if (users.get() > 1) {
+ LOG.warn("Unexpected increment of user count beyond one: " + users.get() + " " + this);
+ }
+ }
+
+ /**
+ * Decrements the user count.
+ */
+ private void release() {
+ if (users.get() > 0) {
+ users.decrementAndGet();
+ } else {
+ LOG.warn("Unexpected attempt to decrement user count of zero: " + users.get() + " " + this);
+ }
+ }
+
+ /**
+ * Communicate to the client that it is no longer in the cache.
+ * The expiration time should be voided to allow the connection to be closed at the first opportunity.
+ */
+ public synchronized void setExpiredFromCache() {
+ if (users.get() != 0) {
+ LOG.warn("Evicted client has non-zero user count: " + users.get());
+ }
+
+ expiredFromCache = true;
+ }
+
+ public boolean isClosed() {
+ return isClosed;
+ }
+
+ /*
+ * Used only for Debugging or testing purposes
+ */
+ public AtomicInteger getUsers() {
+ return users;
+ }
+
+ /**
+ * Make a call to hive meta store and see if the client is still usable. Some calls where the user provides
+ * invalid data renders the client unusable for future use (example: create a table with very long table name)
+ * @return
+ */
+ @Deprecated
+ public boolean isOpen() {
+ try {
+ // Look for an unlikely database name and see if either MetaException or TException is thrown
+ super.getDatabases("NonExistentDatabaseUsedForHealthCheck");
+ } catch (TException e) {
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Decrement the user count and piggyback this to set expiry flag as well, then teardown(), if conditions are met.
+ * This *MUST* be called by anyone who uses this client.
+ */
+ @Override
+ public synchronized void close() {
+ release();
+ tearDownIfUnused();
+ }
+
+ /**
+ * Attempt to tear down the client connection.
+ * The connection will be closed if the following conditions hold:
+ * 1. There are no active user holding the client.
+ * 2. The client has been evicted from the cache.
+ */
+ public synchronized void tearDownIfUnused() {
+ if (users.get() != 0) {
+ LOG.warn("Non-zero user count preventing client tear down: users=" + users.get() + " expired=" + expiredFromCache);
+ }
+
+ if (users.get() == 0 && expiredFromCache) {
+ this.tearDown();
+ }
+ }
+
+ /**
+ * Close the underlying objects irrespective of whether they are in use or not.
+ */
+ public void tearDown() {
+ try {
+ if (!isClosed) {
+ super.close();
+ }
+ isClosed = true;
+ } catch (Exception e) {
+ LOG.warn("Error closing hive metastore client. Ignored.", e);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "HCatClient: thread: " + Thread.currentThread().getId() + " users=" + users.get()
+ + " expired=" + expiredFromCache + " closed=" + isClosed;
+ }
+
+ /**
+ * GC is attempting to destroy the object.
+ * No one references this client anymore, so it can be torn down without worrying about user counts.
+ * @throws Throwable
+ */
+ @Override
+ protected void finalize() throws Throwable {
+ if (users.get() != 0) {
+ LOG.warn("Closing client with non-zero user count: users=" + users.get() + " expired=" + expiredFromCache);
+ }
+
+ try {
+ this.tearDown();
+ } finally {
+ super.finalize();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreUtils.java
index a66c135..75a7201 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreUtils.java
@@ -18,9 +18,13 @@
package org.apache.hadoop.hive.metastore;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -42,6 +46,7 @@ import org.apache.hive.common.util.ReflectionUtil;
public class HiveMetaStoreUtils {
protected static final Logger LOG = LoggerFactory.getLogger("hive.log");
+ private static volatile HiveClientCache hiveClientCache;
/**
* getDeserializer
@@ -210,4 +215,34 @@ public class HiveMetaStoreUtils {
"generated by TypeInfoUtils.getFieldSchemaFromTypeInfo");
}
+ /**
+ * Get or create a hive client depending on whether it exits in cache or not
+ * @param hiveConf The hive configuration
+ * @return the client
+ * @throws MetaException When HiveMetaStoreClient couldn't be created
+ * @throws IOException
+ */
+ public static IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf)
+ throws MetaException, IOException {
+
+ if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.METASTORE_CLIENT_CACHE_ENABLED)){
+ // If cache is disabled, don't use it.
+ return HiveClientCache.getNonCachedHiveMetastoreClient(hiveConf);
+ }
+
+ // Singleton behaviour: create the cache instance if required.
+ if (hiveClientCache == null) {
+ synchronized (IMetaStoreClient.class) {
+ if (hiveClientCache == null) {
+ hiveClientCache = new HiveClientCache(hiveConf);
+ }
+ }
+ }
+ try {
+ return hiveClientCache.get(hiveConf);
+ } catch (LoginException e) {
+ throw new IOException("Couldn't create hiveMetaStoreClient, Error getting UGI for user", e);
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hive/blob/bf8d305a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
index 09f8802..0f03dbc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
@@ -231,8 +231,7 @@ public class OrcRecordUpdater implements RecordUpdater {
fs = partitionRoot.getFileSystem(options.getConfiguration());
}
this.fs = fs;
- if (options.getMinimumWriteId() != options.getMaximumWriteId()
- && !options.isWritingBase()) {
+ if (options.getMinimumWriteId() != options.getMaximumWriteId() && !options.isWritingBase()) {
//throw if file already exists as that should never happen
flushLengths = fs.create(OrcAcidUtils.getSideFile(this.path), false, 8,
options.getReporter());
@@ -468,17 +467,20 @@ public class OrcRecordUpdater implements RecordUpdater {
@Override
public void flush() throws IOException {
- // We only support flushes on files with multiple transactions, because
- // flushes create significant overhead in HDFS. Record updaters with a
- // single transaction should be closed rather than flushed.
+ initWriter();
+ // streaming ingest writer with single transaction batch size, in which case the transaction is
+ // either committed or aborted. In either cases we don't need flush length file but we need to
+ // flush intermediate footer to reduce memory pressure. Also with HIVE-19206, streaming writer does
+ // automatic memory management which would require flush of open files without actually closing it.
if (flushLengths == null) {
- throw new IllegalStateException("Attempting to flush a RecordUpdater on "
- + path + " with a single transaction.");
+ // transaction batch size = 1 case
+ writer.writeIntermediateFooter();
+ } else {
+ // transaction batch size > 1 case
+ long len = writer.writeIntermediateFooter();
+ flushLengths.writeLong(len);
+ OrcInputFormat.SHIMS.hflush(flushLengths);
}
- initWriter();
- long len = writer.writeIntermediateFooter();
- flushLengths.writeLong(len);
- OrcInputFormat.SHIMS.hflush(flushLengths);
//multiple transactions only happen for streaming ingest which only allows inserts
assert deleteEventWriter == null : "unexpected delete writer for " + path;
}