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;
   }