You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/07/25 18:27:11 UTC

[02/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
----------------------------------------------------------------------
diff --cc standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index 0000000,54bf3d7..0aab253
mode 000000,100644..100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@@ -1,0 -1,1117 +1,1154 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore.client;
+ 
++import java.net.ProtocolException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ 
+ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+ import org.apache.thrift.TException;
++import org.apache.thrift.protocol.TProtocolException;
+ import org.apache.thrift.transport.TTransportException;
+ 
+ import com.google.common.collect.Lists;
+ 
+ import org.junit.After;
+ import org.junit.AfterClass;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ 
+ import static java.util.stream.Collectors.joining;
+ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertFalse;
+ import static org.junit.Assert.assertNotEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ /**
+  * API tests for HMS client's  alterPartitions methods.
+  */
+ @RunWith(Parameterized.class)
+ @Category(MetastoreCheckinTest.class)
+ public class TestAlterPartitions extends MetaStoreClientTest {
+   private static final int NEW_CREATE_TIME = 123456789;
+   private AbstractMetaStoreService metaStore;
+   private IMetaStoreClient client;
+ 
+   private static final String DB_NAME = "testpartdb";
+   private static final String TABLE_NAME = "testparttable";
+   private static final List<String> PARTCOL_SCHEMA = Lists.newArrayList("yyyy", "mm", "dd");
+ 
+   public TestAlterPartitions(String name, AbstractMetaStoreService metaStore) {
+     this.metaStore = metaStore;
+   }
+ 
+   @Before
+   public void setUp() throws Exception {
+     // Get new client
+     client = metaStore.getClient();
+ 
+     // Clean up the database
+     client.dropDatabase(DB_NAME, true, true, true);
+     metaStore.cleanWarehouseDirs();
+     createDB(DB_NAME);
+   }
+ 
+   @After
+   public void tearDown() throws Exception {
+     try {
+       if (client != null) {
+         try {
+           client.close();
+         } catch (Exception e) {
+           // HIVE-19729: Shallow the exceptions based on the discussion in the Jira
+         }
+       }
+     } finally {
+       client = null;
+     }
+   }
+ 
+   private void createDB(String dbName) throws TException {
+     new DatabaseBuilder().
+             setName(dbName).
+             create(client, metaStore.getConf());
+   }
+ 
+   private Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
+                                        List<String> partCols, boolean setPartitionLevelPrivilages)
+           throws Exception {
+     TableBuilder builder = new TableBuilder()
+             .setDbName(dbName)
+             .setTableName(tableName)
+             .addCol("id", "int")
+             .addCol("name", "string");
+ 
+     partCols.forEach(col -> builder.addPartCol(col, "string"));
+     Table table = builder.build(metaStore.getConf());
+ 
+     if (setPartitionLevelPrivilages) {
+       table.putToParameters("PARTITION_LEVEL_PRIVILEGE", "true");
+     }
+ 
+     client.createTable(table);
+     return table;
+   }
+ 
+   private void addPartition(IMetaStoreClient client, Table table, List<String> values)
+           throws TException {
+     PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(table);
+     values.forEach(val -> partitionBuilder.addValue(val));
+     client.add_partition(partitionBuilder.build(metaStore.getConf()));
+   }
+ 
+   private List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
+           Exception {
+     Table t = createTestTable(client, DB_NAME, TABLE_NAME, PARTCOL_SCHEMA, false);
+     List<List<String>> testValues = Lists.newArrayList(
+             Lists.newArrayList("1999", "01", "02"),
+             Lists.newArrayList("2009", "02", "10"),
+             Lists.newArrayList("2017", "10", "26"),
+             Lists.newArrayList("2017", "11", "27"));
+ 
+     for(List<String> vals : testValues){
+       addPartition(client, t, vals);
+     }
+ 
+     return testValues;
+   }
+ 
+   private static void assertPartitionsHaveCorrectValues(List<Partition> partitions,
+                                     List<List<String>> testValues) throws Exception {
+     assertEquals(testValues.size(), partitions.size());
+     for (int i = 0; i < partitions.size(); ++i) {
+       assertEquals(testValues.get(i), partitions.get(i).getValues());
+     }
+   }
+ 
+   private static void makeTestChangesOnPartition(Partition partition) {
+     partition.getParameters().put("hmsTestParam001", "testValue001");
+     partition.setCreateTime(NEW_CREATE_TIME);
+     partition.setLastAccessTime(NEW_CREATE_TIME);
+     partition.getSd().setLocation(partition.getSd().getLocation()+"/hh=01");
+     partition.getSd().getCols().add(new FieldSchema("newcol", "string", ""));
+   }
+ 
+   private void assertPartitionUnchanged(Partition partition, List<String> testValues,
+                                                List<String> partCols) throws MetaException {
+     assertFalse(partition.getParameters().containsKey("hmsTestParam001"));
+ 
+     List<String> expectedKVPairs = new ArrayList<>();
+     for (int i = 0; i < partCols.size(); ++i) {
+       expectedKVPairs.add(partCols.get(i) + "=" + testValues.get(i));
+     }
+     String partPath = expectedKVPairs.stream().collect(joining("/"));
+     assertTrue(partition.getSd().getLocation().equals(metaStore.getWarehouseRoot()
+         + "/testpartdb.db/testparttable/" + partPath));
+     assertNotEquals(NEW_CREATE_TIME, partition.getCreateTime());
+     assertNotEquals(NEW_CREATE_TIME, partition.getLastAccessTime());
+     assertEquals(2, partition.getSd().getCols().size());
+   }
+ 
+   private void assertPartitionChanged(Partition partition, List<String> testValues,
+                                       List<String> partCols) throws MetaException {
+     assertEquals("testValue001", partition.getParameters().get("hmsTestParam001"));
+ 
+     List<String> expectedKVPairs = new ArrayList<>();
+     for (int i = 0; i < partCols.size(); ++i) {
+       expectedKVPairs.add(partCols.get(i) + "=" + testValues.get(i));
+     }
+     String partPath = expectedKVPairs.stream().collect(joining("/"));
+     assertTrue(partition.getSd().getLocation().equals(metaStore.getWarehouseRoot()
+         + "/testpartdb.db/testparttable/" + partPath + "/hh=01"));
+     assertEquals(NEW_CREATE_TIME, partition.getCreateTime());
+     assertEquals(NEW_CREATE_TIME, partition.getLastAccessTime());
+     assertEquals(3, partition.getSd().getCols().size());
+   }
+ 
+ 
+ 
+   /**
+    * Testing alter_partition(String,String,Partition) ->
+    *         alter_partition_with_environment_context(String,String,Partition,null).
+    */
+   @Test
+   public void testAlterPartition() throws Exception {
+     List<List<String>> testValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition oldPart = oldParts.get(3);
+ 
+     assertPartitionUnchanged(oldPart, testValues.get(3), PARTCOL_SCHEMA);
+     makeTestChangesOnPartition(oldPart);
+ 
+     client.alter_partition(DB_NAME, TABLE_NAME, oldPart);
+ 
+     List<Partition> newParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition newPart = newParts.get(3);
+     assertPartitionChanged(newPart, testValues.get(3), PARTCOL_SCHEMA);
+     assertPartitionsHaveCorrectValues(newParts, testValues);
+ 
+   }
+ 
+   @Test
+   public void otherCatalog() throws TException {
+     String catName = "alter_partition_catalog";
+     Catalog cat = new CatalogBuilder()
+         .setName(catName)
+         .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+         .build();
+     client.createCatalog(cat);
+ 
+     String dbName = "alter_partition_database_in_other_catalog";
+     Database db = new DatabaseBuilder()
+         .setName(dbName)
+         .setCatalogName(catName)
+         .create(client, metaStore.getConf());
+ 
+     String tableName = "table_in_other_catalog";
+     Table table = new TableBuilder()
+         .inDb(db)
+         .setTableName(tableName)
+         .addCol("id", "int")
+         .addCol("name", "string")
+         .addPartCol("partcol", "string")
+         .create(client, metaStore.getConf());
+ 
+     Partition[] parts = new Partition[5];
+     for (int i = 0; i < 5; i++) {
+       parts[i] = new PartitionBuilder()
+           .inTable(table)
+           .addValue("a" + i)
+           .setLocation(MetaStoreTestUtils.getTestWarehouseDir("b" + i))
+           .build(metaStore.getConf());
+     }
+     client.add_partitions(Arrays.asList(parts));
+ 
+     Partition newPart =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a0"));
+     newPart.getParameters().put("test_key", "test_value");
+     client.alter_partition(catName, dbName, tableName, newPart);
+ 
+     Partition fetched =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a0"));
+     Assert.assertEquals(catName, fetched.getCatName());
+     Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+ 
+     newPart =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+     newPart.setLastAccessTime(3);
+     Partition newPart1 =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
+     newPart1.getSd().setLocation(MetaStoreTestUtils.getTestWarehouseDir("somewhere"));
+     client.alter_partitions(catName, dbName, tableName, Arrays.asList(newPart, newPart1));
+     fetched =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+     Assert.assertEquals(catName, fetched.getCatName());
+     Assert.assertEquals(3L, fetched.getLastAccessTime());
+     fetched =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
+     Assert.assertEquals(catName, fetched.getCatName());
+     Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
+ 
+     newPart =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a4"));
+     newPart.getParameters().put("test_key", "test_value");
+     EnvironmentContext ec = new EnvironmentContext();
+     ec.setProperties(Collections.singletonMap("a", "b"));
+     client.alter_partition(catName, dbName, tableName, newPart, ec);
+     fetched =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a4"));
+     Assert.assertEquals(catName, fetched.getCatName());
+     Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+ 
+ 
+     client.dropDatabase(catName, dbName, true, true, true);
+     client.dropCatalog(catName);
+   }
+ 
+   @SuppressWarnings("deprecation")
+   @Test
+   public void deprecatedCalls() throws TException {
+     String tableName = "deprecated_table";
+     Table table = new TableBuilder()
+         .setTableName(tableName)
+         .addCol("id", "int")
+         .addCol("name", "string")
+         .addPartCol("partcol", "string")
+         .create(client, metaStore.getConf());
+ 
+     Partition[] parts = new Partition[5];
+     for (int i = 0; i < 5; i++) {
+       parts[i] = new PartitionBuilder()
+           .inTable(table)
+           .addValue("a" + i)
+           .setLocation(MetaStoreTestUtils.getTestWarehouseDir("a" + i))
+           .build(metaStore.getConf());
+     }
+     client.add_partitions(Arrays.asList(parts));
+ 
+     Partition newPart =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a0"));
+     newPart.getParameters().put("test_key", "test_value");
+     client.alter_partition(DEFAULT_DATABASE_NAME, tableName, newPart);
+ 
+     Partition fetched =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a0"));
+     Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+ 
+     newPart =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a1"));
+     newPart.setLastAccessTime(3);
+     Partition newPart1 =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a2"));
+     newPart1.getSd().setLocation("somewhere");
+     client.alter_partitions(DEFAULT_DATABASE_NAME, tableName, Arrays.asList(newPart, newPart1));
+     fetched =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a1"));
+     Assert.assertEquals(3L, fetched.getLastAccessTime());
+     fetched =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a2"));
+     Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
+ 
+     newPart =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a3"));
+     newPart.setValues(Collections.singletonList("b3"));
+     client.renamePartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a3"), newPart);
+     fetched =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("b3"));
+     Assert.assertEquals(1, fetched.getValuesSize());
+     Assert.assertEquals("b3", fetched.getValues().get(0));
+ 
+     newPart =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a4"));
+     newPart.getParameters().put("test_key", "test_value");
+     EnvironmentContext ec = new EnvironmentContext();
+     ec.setProperties(Collections.singletonMap("a", "b"));
+     client.alter_partition(DEFAULT_DATABASE_NAME, tableName, newPart, ec);
+     fetched =
+         client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a4"));
+     Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionUnknownPartition() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionIncompletePartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionMissingPartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionBogusCatalogName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     client.alter_partition("nosuch", DB_NAME, TABLE_NAME, partitions.get(3));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionNoDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     client.alter_partition("", TABLE_NAME, partitions.get(3));
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionNullDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
 -    client.alter_partition(null, TABLE_NAME, partitions.get(3));
++    try {
++      client.alter_partition(null, TABLE_NAME, partitions.get(3));
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionNoTblName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     client.alter_partition(DB_NAME, "", partitions.get(3));
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionNullTblName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
 -    client.alter_partition(DB_NAME, null, partitions.get(3));
++    try {
++      client.alter_partition(DB_NAME, null, partitions.get(3));
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test
+   public void testAlterPartitionNullPartition() throws Exception {
+     try {
+       createTable4PartColsParts(client);
+       List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+       client.alter_partition(DB_NAME, TABLE_NAME, null);
+       fail("Should have thrown exception");
+     } catch (NullPointerException | TTransportException e) {
+       //TODO: should not throw different exceptions for different HMS deployment types
+     }
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionChangeDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setDbName(DB_NAME+"_changed");
+     client.alter_partition(DB_NAME, TABLE_NAME, partition);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionChangeTableName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setTableName(TABLE_NAME+"_changed");
+     client.alter_partition(DB_NAME, TABLE_NAME, partition);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionChangeValues() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setValues(Lists.newArrayList("1", "2", "3"));
+     client.alter_partition(DB_NAME, TABLE_NAME, partition);
+   }
+ 
+ 
+   /**
+    * Testing alter_partition(String,String,Partition,EnvironmentContext) ->
+    *         alter_partition_with_environment_context(String,String,Partition,EnvironmentContext).
+    */
+   @Test
+   public void testAlterPartitionWithEnvironmentCtx() throws Exception {
+     EnvironmentContext context = new EnvironmentContext();
+     context.setProperties(new HashMap<String, String>(){
+       {
+         put("TestKey", "TestValue");
+       }
+     });
+ 
+     List<List<String>> testValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = oldParts.get(3);
+ 
+     assertPartitionUnchanged(partition, testValues.get(3), PARTCOL_SCHEMA);
+     makeTestChangesOnPartition(partition);
+ 
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, context);
+ 
+     List<Partition> newParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     partition = newParts.get(3);
+     assertPartitionChanged(partition, testValues.get(3), PARTCOL_SCHEMA);
+     assertPartitionsHaveCorrectValues(newParts, testValues);
+ 
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, new EnvironmentContext());
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, null);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionWithEnvironmentCtxUnknownPartition() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionWithEnvironmentCtxIncompletePartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionWithEnvironmentCtxMissingPartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).build(metaStore.getConf());
+     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionWithEnvironmentCtxNoDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     client.alter_partition("", TABLE_NAME, partitions.get(3), new EnvironmentContext());
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionWithEnvironmentCtxNullDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
 -    client.alter_partition(null, TABLE_NAME, partitions.get(3), new EnvironmentContext());
++    try {
++      client.alter_partition(null, TABLE_NAME, partitions.get(3), new EnvironmentContext());
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionWithEnvironmentCtxNoTblName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     client.alter_partition(DB_NAME, "", partitions.get(3), new EnvironmentContext());
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionWithEnvironmentCtxNullTblName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
 -    client.alter_partition(DB_NAME, null, partitions.get(3), new EnvironmentContext());
++    try {
++      client.alter_partition(DB_NAME, null, partitions.get(3), new EnvironmentContext());
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test
+   public void testAlterPartitionWithEnvironmentCtxNullPartition() throws Exception {
+     try {
+       createTable4PartColsParts(client);
+       List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short) -1);
+       client.alter_partition(DB_NAME, TABLE_NAME, null, new EnvironmentContext());
+       fail("Should have thrown exception");
+     } catch (NullPointerException | TTransportException e) {
+       //TODO: should not throw different exceptions for different HMS deployment types
+     }
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionWithEnvironmentCtxChangeDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setDbName(DB_NAME+"_changed");
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionWithEnvironmentCtxChangeTableName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setTableName(TABLE_NAME+"_changed");
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, new EnvironmentContext());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionWithEnvironmentCtxChangeValues() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition partition = partitions.get(3);
+     partition.setValues(Lists.newArrayList("1", "2", "3"));
+     client.alter_partition(DB_NAME, TABLE_NAME, partition, new EnvironmentContext());
+   }
+ 
+ 
+ 
+   /**
+    * Testing
+    *    alter_partitions(String,String,List(Partition)) ->
+    *    alter_partitions_with_environment_context(String,String,List(Partition),null).
+    */
+   @Test
+   public void testAlterPartitions() throws Exception {
+     List<List<String>> testValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     for (int i = 0; i < testValues.size(); ++i) {
+       assertPartitionUnchanged(oldParts.get(i), testValues.get(i), PARTCOL_SCHEMA);
+     }
+     oldParts.forEach(p -> makeTestChangesOnPartition(p));
+ 
+     client.alter_partitions(DB_NAME, TABLE_NAME, oldParts);
+ 
+     List<Partition> newParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     for (int i = 0; i < testValues.size(); ++i) {
+       assertPartitionChanged(oldParts.get(i), testValues.get(i), PARTCOL_SCHEMA);
+     }
+     assertPartitionsHaveCorrectValues(newParts, testValues);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsEmptyPartitionList() throws Exception {
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList());
+   }
+ 
+   @Test
+   public void testAlterPartitionsUnknownPartition() throws Exception {
+     Partition part1 = null;
+     try {
+       createTable4PartColsParts(client);
+       Table t = client.getTable(DB_NAME, TABLE_NAME);
+       PartitionBuilder builder = new PartitionBuilder();
+       Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
+       part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short) -1).get(0);
+       makeTestChangesOnPartition(part1);
+       client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
+       fail("Should have thrown InvalidOperationException");
+     } catch (InvalidOperationException e) {
+       part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short) -1).get(0);
+       assertPartitionUnchanged(part1, part1.getValues(), PARTCOL_SCHEMA);
+     }
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsIncompletePartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
+     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsMissingPartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).build(metaStore.getConf());
+     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsBogusCatalogName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList(part));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsNoDbName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions("", TABLE_NAME, Lists.newArrayList(part));
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionsNullDbName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
 -    client.alter_partitions(null, TABLE_NAME, Lists.newArrayList(part));
++    try {
++      client.alter_partitions(null, TABLE_NAME, Lists.newArrayList(part));
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsNoTblName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, "", Lists.newArrayList(part));
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionsNullTblName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
 -    client.alter_partitions(DB_NAME, null, Lists.newArrayList(part));
++    try {
++      client.alter_partitions(DB_NAME, null, Lists.newArrayList(part));
++      Assert.fail("didn't throw");
++    } catch (TProtocolException | MetaException e) {
++      // By design
++    }
+   }
+ 
+   @Test(expected = NullPointerException.class)
+   public void testAlterPartitionsNullPartition() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, null));
+   }
+ 
+   @Test(expected = NullPointerException.class)
+   public void testAlterPartitionsNullPartitions() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(null, null));
+   }
+ 
+   @Test
+   public void testAlterPartitionsNullPartitionList() throws Exception {
+     try {
+       createTable4PartColsParts(client);
+       Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+       client.alter_partitions(DB_NAME, TABLE_NAME, null);
+       fail("Should have thrown exception");
 -    } catch (NullPointerException | TTransportException e) {
++    } catch (NullPointerException | TTransportException | TProtocolException e) {
+       //TODO: should not throw different exceptions for different HMS deployment types
+     }
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsChangeDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setDbName(DB_NAME+"_changed");
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p));
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsChangeTableName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setTableName(TABLE_NAME+"_changed");
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsChangeValues() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setValues(Lists.newArrayList("1", "2", "3"));
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p));
+   }
+ 
+ 
+ 
+   /**
+    * Testing
+    *    alter_partitions(String,String,List(Partition),EnvironmentContext) ->
+    *    alter_partitions_with_environment_context(String,String,List(Partition),EnvironmentContext).
+    */
+   @Test
+   public void testAlterPartitionsWithEnvironmentCtx() throws Exception {
+     EnvironmentContext context = new EnvironmentContext();
+     context.setProperties(new HashMap<String, String>(){
+       {
+         put("TestKey", "TestValue");
+       }
+     });
+ 
+     List<List<String>> testValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     for (int i = 0; i < testValues.size(); ++i) {
+       assertPartitionUnchanged(oldParts.get(i), testValues.get(i), PARTCOL_SCHEMA);
+     }
+     oldParts.forEach(p -> makeTestChangesOnPartition(p));
+ 
+     client.alter_partitions(DB_NAME, TABLE_NAME, oldParts, context);
+ 
+     List<Partition> newParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     for (int i = 0; i < testValues.size(); ++i) {
+       assertPartitionChanged(oldParts.get(i), testValues.get(i), PARTCOL_SCHEMA);
+     }
+     assertPartitionsHaveCorrectValues(newParts, testValues);
+ 
+     client.alter_partitions(DB_NAME, TABLE_NAME, newParts, new EnvironmentContext());
 -    client.alter_partitions(DB_NAME, TABLE_NAME, newParts, null);
++    client.alter_partitions(DB_NAME, TABLE_NAME, newParts);
+ 
+     for (int i = 0; i < testValues.size(); ++i) {
+       assertPartitionChanged(oldParts.get(i), testValues.get(i), PARTCOL_SCHEMA);
+     }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxEmptyPartitionList() throws Exception {
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(), new EnvironmentContext());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxUnknownPartition() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
+     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
+             new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsWithEnvironmentCtxIncompletePartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
+     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
+             new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsWithEnvironmentCtxMissingPartitionVals() throws Exception {
+     createTable4PartColsParts(client);
+     Table t = client.getTable(DB_NAME, TABLE_NAME);
+     PartitionBuilder builder = new PartitionBuilder();
+     Partition part = builder.inTable(t).build(metaStore.getConf());
+     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
+             new EnvironmentContext());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxBogusCatalogName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
 -    client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext());
++    client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext(),
++        -1, null, -1);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxNoDbName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions("", TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext());
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionsWithEnvironmentCtxNullDbName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
 -    client.alter_partitions(null, TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext());
++    try {
++      client.alter_partitions(null, TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext());
++      Assert.fail("Expected exception");
++    } catch (MetaException | TProtocolException ex) {
++    }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxNoTblName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, "", Lists.newArrayList(part), new EnvironmentContext());
+   }
+ 
 -  @Test(expected = MetaException.class)
++  @Test
+   public void testAlterPartitionsWithEnvironmentCtxNullTblName() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
 -    client.alter_partitions(DB_NAME, null, Lists.newArrayList(part), new EnvironmentContext());
++    try {
++      client.alter_partitions(DB_NAME, null, Lists.newArrayList(part), new EnvironmentContext());
++      Assert.fail("didn't throw");
++    } catch (MetaException | TProtocolException ex) {
++      // By design.
++    }
+   }
+ 
+   @Test(expected = NullPointerException.class)
+   public void testAlterPartitionsWithEnvironmentCtxNullPartition() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, null),
+             new EnvironmentContext());
+   }
+ 
+   @Test(expected = NullPointerException.class)
+   public void testAlterPartitionsWithEnvironmentCtxNullPartitions() throws Exception {
+     createTable4PartColsParts(client);
+     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(null, null),
+             new EnvironmentContext());
+   }
+ 
+   @Test
+   public void testAlterPartitionsWithEnvironmentCtxNullPartitionList() throws Exception {
+     try {
+       createTable4PartColsParts(client);
+       Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+       client.alter_partitions(DB_NAME, TABLE_NAME, null, new EnvironmentContext());
+       fail("Should have thrown exception");
 -    } catch (NullPointerException | TTransportException e) {
++    } catch (NullPointerException | TTransportException | TProtocolException e) {
+       //TODO: should not throw different exceptions for different HMS deployment types
+     }
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsWithEnvironmentCtxChangeDbName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setDbName(DB_NAME+"_changed");
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p), new EnvironmentContext());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAlterPartitionsWithEnvironmentCtxChangeTableName() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setTableName(TABLE_NAME+"_changed");
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p), new EnvironmentContext());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testAlterPartitionsWithEnvironmentCtxChangeValues() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     Partition p = partitions.get(3);
+     p.setValues(Lists.newArrayList("1", "2", "3"));
+     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(p), new EnvironmentContext());
+   }
+ 
+   /**
+    * Testing
+    *    renamePartition(String,String,List(String),Partition) ->
+    *    renamePartition(String,String,List(String),Partition).
+    */
+   @Test
+   public void testRenamePartition() throws Exception {
+ 
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<List<String>> newValues = new ArrayList<>();
+ 
+     List<String> newVal = Lists.newArrayList("2018", "01", "16");
+     newValues.addAll(oldValues.subList(0, 3));
+     newValues.add(newVal);
+ 
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(newVal);
+     makeTestChangesOnPartition(partToRename);
+     client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
+ 
+     List<Partition> newParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+     assertPartitionsHaveCorrectValues(newParts, newValues);
+ 
+ 
+     //Asserting other partition parameters can also be changed, but not the location
+     assertFalse(newParts.get(3).getSd().getLocation().endsWith("hh=01"));
+     assertEquals(3, newParts.get(3).getSd().getCols().size());
+     assertEquals("testValue001", newParts.get(3).getParameters().get("hmsTestParam001"));
+     assertEquals(NEW_CREATE_TIME, newParts.get(3).getCreateTime());
+     assertEquals(NEW_CREATE_TIME, newParts.get(3).getLastAccessTime());
+ 
+ 
+ 
+     assertTrue(client.listPartitions(DB_NAME, TABLE_NAME, oldValues.get(3), (short)-1).isEmpty());
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionTargetAlreadyExisting() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), oldParts.get(2));
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionNoSuchOldPartition() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList("1", "2", ""), partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionNullTableInPartition() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     partToRename.setTableName(null);
+     client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList("2017", "11", "27"),
+             partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionNullDbInPartition() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     partToRename.setDbName(null);
+     client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList("2017", "11", "27"),
+             partToRename);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionEmptyOldPartList() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList(), partToRename);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionNullOldPartList() throws Exception {
+     createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, TABLE_NAME, null, partToRename);
+   }
+ 
+   @Test
+   public void testRenamePartitionNullNewPart() throws Exception {
+     try {
+       List<List<String>> oldValues = createTable4PartColsParts(client);
+       List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short) -1);
+ 
+       Partition partToRename = oldParts.get(3);
+       partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+       client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), null);
+     } catch (NullPointerException | TTransportException e) {
+     }
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionBogusCatalogName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
 -    client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
++    client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), partToRename, -1, null);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionNoDbName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition("", TABLE_NAME, oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionNoTblName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, "", oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionNullDbName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(null, TABLE_NAME, oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionNullTblName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     client.renamePartition(DB_NAME, null, oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionChangeTblName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     partToRename.setTableName(TABLE_NAME + "_2");
+     client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testRenamePartitionChangeDbName() throws Exception {
+     List<List<String>> oldValues = createTable4PartColsParts(client);
+     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+ 
+     Partition partToRename = oldParts.get(3);
+     partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+     partToRename.setDbName(DB_NAME + "_2");
+     client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
+   }
+ 
+   @Test(expected = InvalidOperationException.class)
+   public void testRenamePartitionNoTable() throws Exception {
+     client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList("2018", "01", "16"),
+             new Partition());
+   }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
----------------------------------------------------------------------
diff --cc standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
index 0000000,8ce8531..462584a
mode 000000,100644..100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
@@@ -1,0 -1,594 +1,600 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.client;
+ 
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.stream.Collectors;
+ 
+ import org.apache.commons.lang.StringUtils;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+ import org.apache.hadoop.hive.metastore.TableType;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+ import org.apache.thrift.TException;
+ import org.apache.thrift.transport.TTransportException;
+ import org.junit.After;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ 
+ import com.google.common.collect.Lists;
+ 
+ /**
+  * Tests for appending partitions.
+  */
+ @RunWith(Parameterized.class)
+ @Category(MetastoreCheckinTest.class)
+ public class TestAppendPartitions extends MetaStoreClientTest {
+   private AbstractMetaStoreService metaStore;
+   private IMetaStoreClient client;
+ 
+   private static final String DB_NAME = "test_append_part_db";
+   private static Table tableWithPartitions;
+   private static Table externalTable;
+   private static Table tableNoPartColumns;
+   private static Table tableView;
+ 
+   public TestAppendPartitions(String name, AbstractMetaStoreService metaStore) {
+     this.metaStore = metaStore;
+   }
+ 
+   @Before
+   public void setUp() throws Exception {
+     // Get new client
+     client = metaStore.getClient();
+ 
+     // Clean up the database
+     client.dropDatabase(DB_NAME, true, true, true);
+     metaStore.cleanWarehouseDirs();
+     new DatabaseBuilder()
+         .setName(DB_NAME)
+         .create(client, metaStore.getConf());
+ 
+     tableWithPartitions = createTableWithPartitions();
+     externalTable = createExternalTable();
+     tableNoPartColumns = createTableNoPartitionColumns();
+     tableView = createView();
+   }
+ 
+   @After
+   public void tearDown() throws Exception {
+     try {
+       if (client != null) {
+         try {
+           client.close();
+         } catch (Exception e) {
+           // HIVE-19729: Shallow the exceptions based on the discussion in the Jira
+         }
+       }
+     } finally {
+       client = null;
+     }
+   }
+ 
+   // Tests for Partition appendPartition(String tableName, String dbName, List<String> partVals) method
+ 
+   @Test
+   public void testAppendPartition() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     Table table = tableWithPartitions;
+ 
+     Partition appendedPart =
+         client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+ 
+     Assert.assertNotNull(appendedPart);
+     Partition partition =
+         client.getPartition(table.getDbName(), table.getTableName(), partitionValues);
++    appendedPart.setWriteId(partition.getWriteId());
+     Assert.assertEquals(partition, appendedPart);
+     verifyPartition(partition, table, partitionValues, "year=2017/month=may");
+     verifyPartitionNames(table, Lists.newArrayList("year=2017/month=march", "year=2017/month=april",
+         "year=2018/month=march", "year=2017/month=may"));
+   }
+ 
+   @Test
+   public void testAppendPartitionToExternalTable() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     Table table = externalTable;
+ 
+     Partition appendedPart =
+         client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+ 
+     Assert.assertNotNull(appendedPart);
+     Partition partition =
+         client.getPartition(table.getDbName(), table.getTableName(), partitionValues);
++    appendedPart.setWriteId(partition.getWriteId());
+     Assert.assertEquals(partition, appendedPart);
+     verifyPartition(partition, table, partitionValues, "year=2017/month=may");
+     verifyPartitionNames(table, Lists.newArrayList("year=2017/month=may"));
+   }
+ 
+   @Test
+   public void testAppendPartitionMultiplePartitions() throws Exception {
+ 
+     List<String> partitionValues1 = Lists.newArrayList("2017", "may");
+     List<String> partitionValues2 = Lists.newArrayList("2018", "may");
+     List<String> partitionValues3 = Lists.newArrayList("2017", "june");
+ 
+     Table table = tableWithPartitions;
+ 
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues1);
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues2);
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues3);
+ 
+     verifyPartitionNames(table,
+         Lists.newArrayList("year=2017/month=may", "year=2018/month=may", "year=2017/month=june",
+             "year=2017/month=march", "year=2017/month=april", "year=2018/month=march"));
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionToTableWithoutPartCols() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     Table table = tableNoPartColumns;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionToView() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     Table table = tableView;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = AlreadyExistsException.class)
+   public void testAppendPartitionAlreadyExists() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "april");
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionNonExistingDB() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition("nonexistingdb", tableWithPartitions.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionNonExistingTable() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition(tableWithPartitions.getDbName(), "nonexistingtable", partitionValues);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionEmptyDB() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition("", tableWithPartitions.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionEmptyTable() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition(tableWithPartitions.getDbName(), "", partitionValues);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionNullDB() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition(null, tableWithPartitions.getTableName(), partitionValues);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionNullTable() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2017", "may");
+     client.appendPartition(tableWithPartitions.getDbName(), null, partitionValues);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionEmptyPartValues() throws Exception {
+ 
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), new ArrayList<>());
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartitionNullPartValues() throws Exception {
+ 
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), (List<String>) null);
+   }
+ 
+   @Test
+   public void testAppendPartitionLessPartValues() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2019");
+     Table table = tableWithPartitions;
+ 
+     try {
+       client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+       Assert.fail("Exception should have been thrown.");
+     } catch (MetaException e) {
+       // Expected exception
+     }
+     verifyPartitionNames(table, Lists.newArrayList("year=2017/month=march", "year=2017/month=april",
+         "year=2018/month=march"));
+     String partitionLocation = table.getSd().getLocation() + "/year=2019";
+     Assert.assertFalse(metaStore.isPathExists(new Path(partitionLocation)));
+   }
+ 
+   @Test
+   public void testAppendPartitionMorePartValues() throws Exception {
+ 
+     List<String> partitionValues = Lists.newArrayList("2019", "march", "12");
+     Table table = tableWithPartitions;
+ 
+     try {
+       client.appendPartition(table.getDbName(), table.getTableName(), partitionValues);
+       Assert.fail("Exception should have been thrown.");
+     } catch (MetaException e) {
+       // Expected exception
+     }
+     verifyPartitionNames(table, Lists.newArrayList("year=2017/month=march", "year=2017/month=april",
+         "year=2018/month=march"));
+     String partitionLocation = tableWithPartitions.getSd().getLocation() + "/year=2019";
+     Assert.assertFalse(metaStore.isPathExists(new Path(partitionLocation)));
+   }
+ 
+   // Tests for Partition appendPartition(String tableName, String dbName, String name) method
+ 
+   @Test
+   public void testAppendPart() throws Exception {
+ 
+     Table table = tableWithPartitions;
+     String partitionName = "year=2017/month=may";
+ 
+     Partition appendedPart =
+         client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+ 
+     Assert.assertNotNull(appendedPart);
+     Partition partition = client.getPartition(table.getDbName(), table.getTableName(),
+         getPartitionValues(partitionName));
++    appendedPart.setWriteId(partition.getWriteId());
+     Assert.assertEquals(partition, appendedPart);
+     verifyPartition(partition, table, getPartitionValues(partitionName), partitionName);
+     verifyPartitionNames(table, Lists.newArrayList("year=2017/month=march", "year=2017/month=april",
+         "year=2018/month=march", partitionName));
+   }
+ 
+   @Test
+   public void testAppendPartToExternalTable() throws Exception {
+ 
+     Table table = externalTable;
+     String partitionName = "year=2017/month=may";
+ 
+     Partition appendedPart =
+         client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+ 
+     Assert.assertNotNull(appendedPart);
+     Partition partition = client.getPartition(table.getDbName(), table.getTableName(),
+         getPartitionValues(partitionName));
++    appendedPart.setWriteId(partition.getWriteId());
+     Assert.assertEquals(partition, appendedPart);
+     verifyPartition(partition, table, getPartitionValues(partitionName), partitionName);
+     verifyPartitionNames(table, Lists.newArrayList(partitionName));
+   }
+ 
+   @Test
+   public void testAppendPartMultiplePartitions() throws Exception {
+ 
+     String partitionName1 = "year=2017/month=may";
+     String partitionName2 = "year=2018/month=may";
+     String partitionName3 = "year=2017/month=june";
+     Table table = tableWithPartitions;
+ 
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName1);
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName2);
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName3);
+ 
+     verifyPartitionNames(table, Lists.newArrayList(partitionName1, partitionName2, partitionName3,
+         "year=2017/month=march", "year=2017/month=april", "year=2018/month=march"));
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartToTableWithoutPartCols() throws Exception {
+ 
+     String partitionName = "year=2017/month=may";
+     Table table = tableNoPartColumns;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartToView() throws Exception {
+ 
+     String partitionName = "year=2017/month=may";
+     Table table = tableView;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = AlreadyExistsException.class)
+   public void testAppendPartAlreadyExists() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartNonExistingDB() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition("nonexistingdb", tableWithPartitions.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartNonExistingTable() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition(tableWithPartitions.getDbName(), "nonexistingtable", partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartEmptyDB() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition("", tableWithPartitions.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartEmptyTable() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition(tableWithPartitions.getDbName(), "", partitionName);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartNullDB() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition(null, tableWithPartitions.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartNullTable() throws Exception {
+ 
+     String partitionName = "year=2017/month=april";
+     client.appendPartition(tableWithPartitions.getDbName(), null, partitionName);
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartEmptyPartName() throws Exception {
+ 
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), "");
+   }
+ 
+   @Test(expected = MetaException.class)
+   public void testAppendPartNullPartName() throws Exception {
+ 
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), (String) null);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartLessPartValues() throws Exception {
+ 
+     String partitionName = "year=2019";
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test
+   public void testAppendPartMorePartValues() throws Exception {
+ 
+     String partitionName = "year=2019/month=march/day=12";
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartInvalidPartName() throws Exception {
+ 
+     String partitionName = "invalidpartname";
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartWrongColumnInPartName() throws Exception {
+ 
+     String partitionName = "year=2019/honap=march";
+     Table table = tableWithPartitions;
+     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
+   }
+ 
+   @Test
+   public void otherCatalog() throws TException {
+     String catName = "append_partition_catalog";
+     Catalog cat = new CatalogBuilder()
+         .setName(catName)
+         .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+         .build();
+     client.createCatalog(cat);
+ 
+     String dbName = "append_partition_database_in_other_catalog";
+     Database db = new DatabaseBuilder()
+         .setName(dbName)
+         .setCatalogName(catName)
+         .create(client, metaStore.getConf());
+ 
+     String tableName = "table_in_other_catalog";
+     new TableBuilder()
+         .inDb(db)
+         .setTableName(tableName)
+         .addCol("id", "int")
+         .addCol("name", "string")
+         .addPartCol("partcol", "string")
+         .create(client, metaStore.getConf());
+ 
+     Partition created =
+         client.appendPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+     Assert.assertEquals(1, created.getValuesSize());
+     Assert.assertEquals("a1", created.getValues().get(0));
+     Partition fetched =
+         client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
++    created.setWriteId(fetched.getWriteId());
+     Assert.assertEquals(created, fetched);
+ 
+     created = client.appendPartition(catName, dbName, tableName, "partcol=a2");
+     Assert.assertEquals(1, created.getValuesSize());
+     Assert.assertEquals("a2", created.getValues().get(0));
+     fetched = client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
++    created.setWriteId(fetched.getWriteId());
+     Assert.assertEquals(created, fetched);
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionBogusCatalog() throws Exception {
+     client.appendPartition("nosuch", DB_NAME, tableWithPartitions.getTableName(),
+         Lists.newArrayList("2017", "may"));
+   }
+ 
+   @Test(expected = InvalidObjectException.class)
+   public void testAppendPartitionByNameBogusCatalog() throws Exception {
+     client.appendPartition("nosuch", DB_NAME, tableWithPartitions.getTableName(),
+         "year=2017/month=april");
+   }
+ 
+   // Helper methods
+ 
+   private Table createTableWithPartitions() throws Exception {
+     Table table = createTable("test_append_part_table_with_parts", getYearAndMonthPartCols(), null,
+         TableType.MANAGED_TABLE.name(),
+         metaStore.getWarehouseRoot() + "/test_append_part_table_with_parts");
+     createPartition(table, Lists.newArrayList("2017", "march"));
+     createPartition(table, Lists.newArrayList("2017", "april"));
+     createPartition(table, Lists.newArrayList("2018", "march"));
+     return table;
+   }
+ 
+   private Table createTableNoPartitionColumns() throws Exception {
+     Table table = createTable("test_append_part_table_no_part_columns", null, null, "MANAGED_TABLE",
+         metaStore.getWarehouseRoot() + "/test_append_part_table_no_part_columns");
+     return table;
+   }
+ 
+   private Table createExternalTable() throws Exception {
+     Map<String, String> tableParams = new HashMap<>();
+     tableParams.put("EXTERNAL", "TRUE");
+     Table table = createTable("test_append_part_external_table", getYearAndMonthPartCols(),
+         tableParams, TableType.EXTERNAL_TABLE.name(),
+         metaStore.getWarehouseRoot() + "/test_append_part_external_table");
+     return table;
+   }
+ 
+   private Table createView() throws Exception {
+     Table table = createTable("test_append_part_table_view", getYearAndMonthPartCols(), null,
+         TableType.VIRTUAL_VIEW.name(), null);
+     return table;
+   }
+ 
+   private Table createTable(String tableName, List<FieldSchema> partCols, Map<String,
+       String> tableParams, String tableType, String location) throws Exception {
+     new TableBuilder()
+         .setDbName(DB_NAME)
+         .setTableName(tableName)
+         .addCol("test_id", "int", "test col id")
+         .addCol("test_value", "string", "test col value")
+         .setPartCols(partCols)
+         .setTableParams(tableParams)
+         .setType(tableType)
+         .setLocation(location)
+         .create(client, metaStore.getConf());
+     return client.getTable(DB_NAME, tableName);
+   }
+ 
+   private void createPartition(Table table, List<String> values) throws Exception {
+     new PartitionBuilder()
+         .inTable(table)
+         .setValues(values)
+         .addToTable(client, metaStore.getConf());
+   }
+ 
+   private static List<FieldSchema> getYearAndMonthPartCols() {
+     List<FieldSchema> cols = new ArrayList<>();
+     cols.add(new FieldSchema("year", "string", "year part col"));
+     cols.add(new FieldSchema("month", "string", "month part col"));
+     return cols;
+   }
+ 
+   private static List<String> getPartitionValues(String partitionsName) {
+     List<String> values = new ArrayList<>();
+     if (StringUtils.isEmpty(partitionsName)) {
+       return values;
+     }
+     values = Arrays.stream(partitionsName.split("/")).map(v -> v.split("=")[1])
+         .collect(Collectors.toList());
+     return values;
+   }
+ 
+   private void verifyPartition(Partition partition, Table table, List<String> expectedPartValues,
+       String partitionName) throws Exception {
+     Assert.assertEquals(table.getTableName(), partition.getTableName());
+     Assert.assertEquals(table.getDbName(), partition.getDbName());
+     Assert.assertEquals(expectedPartValues, partition.getValues());
+     Assert.assertNotEquals(0, partition.getCreateTime());
+     Assert.assertEquals(0, partition.getLastAccessTime());
+     Assert.assertEquals(1, partition.getParameters().size());
+     Assert.assertTrue(partition.getParameters().containsKey("transient_lastDdlTime"));
+     StorageDescriptor partitionSD = partition.getSd();
+     Assert.assertEquals(table.getSd().getLocation() + "/" + partitionName,
+         partitionSD.getLocation());
+     partition.getSd().setLocation(table.getSd().getLocation());
+     Assert.assertEquals(table.getSd(), partitionSD);
+     Assert.assertTrue(metaStore.isPathExists(new Path(partitionSD.getLocation())));
+   }
+ 
+   private void verifyPartitionNames(Table table, List<String> expectedPartNames) throws Exception {
+     List<String> partitionNames =
+         client.listPartitionNames(table.getDbName(), table.getTableName(), (short) -1);
+     Assert.assertEquals(expectedPartNames.size(), partitionNames.size());
+     Assert.assertTrue(partitionNames.containsAll(expectedPartNames));
+   }
+ }