You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by jd...@apache.org on 2016/12/08 16:54:40 UTC

kudu git commit: Revert "KUDU-861 Support changing default, storage attributes"

Repository: kudu
Updated Branches:
  refs/heads/master 839bd6f9a -> fe4f157b1


Revert "KUDU-861 Support changing default, storage attributes"

This reverts commit e16a541aedc171eadb738e969a48852ba9ed2909, new clients don't work
with old servers.

Change-Id: Ia15db5407af681a412763681bc4fd43e99968b0b
Reviewed-on: http://gerrit.cloudera.org:8080/5420
Reviewed-by: Will Berkeley <wd...@gmail.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: fe4f157b1348d24a215a9e8135c9b94ed0128e71
Parents: 839bd6f
Author: Jean-Daniel Cryans <jd...@apache.org>
Authored: Thu Dec 8 08:31:11 2016 -0800
Committer: Jean-Daniel Cryans <jd...@apache.org>
Committed: Thu Dec 8 16:53:46 2016 +0000

----------------------------------------------------------------------
 .../apache/kudu/client/AlterTableOptions.java   | 108 +---------
 .../org/apache/kudu/client/ProtobufHelper.java  |  41 +---
 .../org/apache/kudu/client/TestKuduTable.java   | 100 +---------
 src/kudu/cfile/rle_block.h                      |   2 +-
 src/kudu/client/client-test.cc                  | 195 ++++---------------
 src/kudu/client/schema.cc                       |  54 +----
 src/kudu/client/schema.h                        |   5 -
 src/kudu/client/table_alterer-internal.cc       |  32 ++-
 src/kudu/client/value-internal.h                |   4 -
 src/kudu/client/value.cc                        |  18 --
 src/kudu/common/common.proto                    |  12 --
 src/kudu/common/schema.cc                       |  85 ++------
 src/kudu/common/schema.h                        |  41 +---
 src/kudu/common/wire_protocol.cc                |  48 -----
 src/kudu/common/wire_protocol.h                 |   8 -
 .../alter_table-randomized-test.cc              | 109 +----------
 src/kudu/integration-tests/alter_table-test.cc  | 110 -----------
 src/kudu/master/catalog_manager.cc              |  27 +--
 src/kudu/master/master.proto                    |   9 +-
 19 files changed, 94 insertions(+), 914 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
index 3827ed0..31617aa 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/AlterTableOptions.java
@@ -17,15 +17,11 @@
 
 package org.apache.kudu.client;
 
-import static org.apache.kudu.ColumnSchema.CompressionAlgorithm;
-import static org.apache.kudu.ColumnSchema.Encoding;
 import static org.apache.kudu.master.Master.AlterTableRequestPB;
 
 import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
 
 import org.apache.kudu.ColumnSchema;
-import org.apache.kudu.Common;
 import org.apache.kudu.Type;
 import org.apache.kudu.annotations.InterfaceAudience;
 import org.apache.kudu.annotations.InterfaceStability;
@@ -125,107 +121,9 @@ public class AlterTableOptions {
    */
   public AlterTableOptions renameColumn(String oldName, String newName) {
     AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(oldName).setNewName(newName));
-    step.setAlterColumn(alterBuilder);
-    return this;
-  }
-
-  /**
-   * Remove the default value for a column.
-   * @param name name of the column
-   * @return this instance
-   */
-  public AlterTableOptions removeDefault(String name) {
-    AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(name).setRemoveDefault(true));
-    step.setAlterColumn(alterBuilder);
-    return this;
-  }
-
-  /**
-   * Change the default value for a column. `newDefault` must not be null or
-   * else throws {@link IllegalArgumentException}.
-   * @param name name of the column
-   * @param newDefault the new default value
-   * @return this instance
-   */
-  public AlterTableOptions changeDefault(String name, Object newDefault) {
-    if (newDefault == null) {
-      throw new IllegalArgumentException("newDefault cannot be null: " +
-          "use removeDefault to clear a default value");
-    }
-
-    ByteString defaultByteString = ProtobufHelper.objectToByteStringNoType(name, newDefault);
-    AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(name)
-            .setDefaultValue(defaultByteString));
-    step.setAlterColumn(alterBuilder);
-    return this;
-  }
-
-  /**
-   * Change the block size of a column's storage. A nonpositive value indicates
-   * a server-side default.
-   * @param name name of the column
-   * @param blockSize the new block size
-   * @return this instance
-   */
-  public AlterTableOptions changeDesiredBlockSize(String name, int blockSize) {
-    AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(name).setBlockSize(blockSize));
-    step.setAlterColumn(alterBuilder);
-    return this;
-  }
-
-  /**
-   * Change the encoding used for a column.
-   * @param name name of the column
-   * @param encoding the new encoding
-   * @return this instance
-   */
-  public AlterTableOptions changeEncoding(String name, Encoding encoding) {
-    AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(name)
-            .setEncoding(encoding.getInternalPbType()));
-    step.setAlterColumn(alterBuilder);
-    return this;
-  }
-
-  /**
-   * Change the compression used for a column.
-   * @param name the name of the column
-   * @param ca the new compression algorithm
-   * @return this instance
-   */
-  public AlterTableOptions changeCompressionAlgorithm(String name, CompressionAlgorithm ca) {
-    AlterTableRequestPB.Step.Builder step = pb.addAlterSchemaStepsBuilder();
-    step.setType(AlterTableRequestPB.StepType.ALTER_COLUMN);
-    AlterTableRequestPB.AlterColumn.Builder alterBuilder =
-        AlterTableRequestPB.AlterColumn.newBuilder();
-    alterBuilder.setDelta(
-        Common.ColumnSchemaDeltaPB.newBuilder().setName(name)
-            .setCompression(ca.getInternalPbType()));
-    step.setAlterColumn(alterBuilder);
+    step.setType(AlterTableRequestPB.StepType.RENAME_COLUMN);
+    step.setRenameColumn(AlterTableRequestPB.RenameColumn.newBuilder().setOldName(oldName)
+        .setNewName(newName));
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java b/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
index 76e6be6..5831e25 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/ProtobufHelper.java
@@ -17,7 +17,6 @@
 
 package org.apache.kudu.client;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -237,47 +236,9 @@ public class ProtobufHelper {
   }
 
   /**
-   * Serializes an object based on its Java type. Used for Alter Column
-   * operations where the column's type is not available. `value` must be
-   * a Kudu-compatible type or else throw {@link IllegalArgumentException}.
-   *
-   * @param colName the name of the column (for the error message)
-   * @param value the value to serialize
-   * @return the serialized object
-   */
-  protected static ByteString objectToByteStringNoType(String colName, Object value) {
-    byte[] bytes;
-    if (value instanceof Boolean) {
-      bytes = Bytes.fromBoolean((Boolean) value);
-    } else if (value instanceof Byte) {
-      bytes = new byte[] {(Byte) value};
-    } else if (value instanceof Short) {
-      bytes = Bytes.fromShort((Short) value);
-    } else if (value instanceof Integer) {
-      bytes = Bytes.fromInt((Integer) value);
-    } else if (value instanceof Long) {
-      bytes = Bytes.fromLong((Long) value);
-    } else if (value instanceof String) {
-      bytes = ((String) value).getBytes(Charsets.UTF_8);
-    } else if (value instanceof byte[]) {
-      bytes = (byte[]) value;
-    } else if (value instanceof ByteBuffer) {
-      bytes = ((ByteBuffer) value).array();
-    } else if (value instanceof Float) {
-      bytes = Bytes.fromFloat((Float) value);
-    } else if (value instanceof Double) {
-      bytes = Bytes.fromDouble((Double) value);
-    } else {
-      throw new IllegalArgumentException("The default value provided for " +
-          "column " + colName + " is of class " + value.getClass().getName() +
-          " which does not map to a supported Kudu type");
-    }
-    return ZeroCopyLiteralByteString.wrap(bytes);
-  }
-
-  /**
    * Convert a {@link com.google.common.net.HostAndPort} to
    *     {@link org.apache.kudu.Common.HostPortPB}
+   * protobuf message for serialization.
    * @param hostAndPort The host and port object. Both host and port must be specified.
    * @return An initialized HostPortPB object.
    */

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
index 8fa9ad0..09ef520 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduTable.java
@@ -16,12 +16,7 @@
 // under the License.
 package org.apache.kudu.client;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -52,98 +47,6 @@ public class TestKuduTable extends BaseKuduTest {
     BaseKuduTest.setUpBeforeClass();
   }
 
-  @Test
-  public void testAlterColumn() throws Exception {
-    // Used a simplified schema because basicSchema has extra columns that make the asserts verbose.
-    String tableName = name.getMethodName() + System.currentTimeMillis();
-    List<ColumnSchema> columns = ImmutableList.of(
-        new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32).key(true).build(),
-        new ColumnSchema.ColumnSchemaBuilder("value", Type.STRING)
-            .nullable(true)
-            .desiredBlockSize(4096)
-            .encoding(ColumnSchema.Encoding.PLAIN_ENCODING)
-            .compressionAlgorithm(ColumnSchema.CompressionAlgorithm.NO_COMPRESSION)
-            .build());
-    KuduTable table = createTable(tableName, new Schema(columns), getBasicCreateTableOptions());
-    KuduSession session = syncClient.newSession();
-    // Insert a row before a default is defined and check the value is NULL.
-    insertDefaultRow(table, session, 0);
-    List<String> rows = scanTableToStrings(table);
-    assertEquals("wrong number of rows", 1, rows.size());
-    assertEquals("wrong row", "INT32 key=0, STRING value=NULL", rows.get(0));
-
-    // Add a default, checking new rows see the new default and old rows remain the same.
-    AlterTableOptions ato = new AlterTableOptions().changeDefault("value", "pizza");
-    submitAlterAndCheck(ato, tableName);
-
-    insertDefaultRow(table, session, 1);
-    rows = scanTableToStrings(table);
-    assertEquals("wrong number of rows", 2, rows.size());
-    assertEquals("wrong row", "INT32 key=0, STRING value=NULL", rows.get(0));
-    assertEquals("wrong row", "INT32 key=1, STRING value=pizza", rows.get(1));
-
-    // Change the default, checking new rows see the new default and old rows remain the same.
-    ato = new AlterTableOptions().changeDefault("value", "taco");
-    submitAlterAndCheck(ato, tableName);
-
-    insertDefaultRow(table, session, 2);
-
-    rows = scanTableToStrings(table);
-    assertEquals("wrong number of rows", 3, rows.size());
-    assertEquals("wrong row", "INT32 key=0, STRING value=NULL", rows.get(0));
-    assertEquals("wrong row", "INT32 key=1, STRING value=pizza", rows.get(1));
-    assertEquals("wrong row", "INT32 key=2, STRING value=taco", rows.get(2));
-
-    // Remove the default, checking that new rows default to NULL and old rows remain the same.
-    ato = new AlterTableOptions().removeDefault("value");
-    submitAlterAndCheck(ato, tableName);
-
-    insertDefaultRow(table, session, 3);
-
-    rows = scanTableToStrings(table);
-    assertEquals("wrong number of rows", 4, rows.size());
-    assertEquals("wrong row", "INT32 key=0, STRING value=NULL", rows.get(0));
-    assertEquals("wrong row", "INT32 key=1, STRING value=pizza", rows.get(1));
-    assertEquals("wrong row", "INT32 key=2, STRING value=taco", rows.get(2));
-    assertEquals("wrong row", "INT32 key=3, STRING value=NULL", rows.get(3));
-
-    // Change the column storage attributes.
-    assertEquals("wrong block size",
-        4096,
-        table.getSchema().getColumn("value").getDesiredBlockSize());
-    assertEquals("wrong encoding",
-        ColumnSchema.Encoding.PLAIN_ENCODING,
-        table.getSchema().getColumn("value").getEncoding());
-    assertEquals("wrong compression algorithm",
-        ColumnSchema.CompressionAlgorithm.NO_COMPRESSION,
-        table.getSchema().getColumn("value").getCompressionAlgorithm());
-
-    ato = new AlterTableOptions().changeDesiredBlockSize("value", 8192)
-        .changeEncoding("value", ColumnSchema.Encoding.DICT_ENCODING)
-        .changeCompressionAlgorithm("value", ColumnSchema.CompressionAlgorithm.SNAPPY);
-    submitAlterAndCheck(ato, tableName);
-
-    KuduTable reopenedTable = syncClient.openTable(tableName);
-    assertEquals("wrong block size post alter",
-        8192,
-        reopenedTable.getSchema().getColumn("value").getDesiredBlockSize());
-    assertEquals("wrong encoding post alter",
-        ColumnSchema.Encoding.DICT_ENCODING,
-        reopenedTable.getSchema().getColumn("value").getEncoding());
-    assertEquals("wrong compression algorithm post alter",
-        ColumnSchema.CompressionAlgorithm.SNAPPY,
-        reopenedTable.getSchema().getColumn("value").getCompressionAlgorithm());
-  }
-
-  private void insertDefaultRow(KuduTable table, KuduSession session, int key)
-      throws Exception {
-    Insert insert = table.newInsert();
-    PartialRow row = insert.getRow();
-    row.addInt("key", key);
-    // Omit value.
-    session.apply(insert);
-  }
-
   @Test(timeout = 100000)
   public void testAlterTable() throws Exception {
     String tableName = name.getMethodName() + System.currentTimeMillis();
@@ -180,6 +83,7 @@ public class TestKuduTable extends BaseKuduTest {
               (System.currentTimeMillis() * 1000));
       submitAlterAndCheck(ato, tableName);
 
+
       // Try altering a table that doesn't exist.
       String nonExistingTableName = "table_does_not_exist";
       try {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/cfile/rle_block.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/rle_block.h b/src/kudu/cfile/rle_block.h
index 3cecf97..c7702f1 100644
--- a/src/kudu/cfile/rle_block.h
+++ b/src/kudu/cfile/rle_block.h
@@ -330,7 +330,7 @@ class RleIntBlockDecoder final : public BlockDecoder {
 
   virtual void SeekToPositionInBlock(uint pos) OVERRIDE {
     CHECK(parsed_) << "Must call ParseHeader()";
-    CHECK(pos < num_elems_ || (num_elems_ == 0 && pos == 0))
+    CHECK_LT(pos, num_elems_)
         << "Tried to seek to " << pos << " which is >= number of elements ("
         << num_elems_ << ") in the block!.";
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index 4a6fcdb..e467164 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -352,7 +352,7 @@ class ClientTest : public KuduTest {
   void DoTestScanResourceMetrics() {
     KuduScanner scanner(client_table_.get());
     string tablet_id = GetFirstTabletId(client_table_.get());
-    // Flush to ensure we scan disk later.
+    // flush to ensure we scan disk later
     FlushTablet(tablet_id);
     ASSERT_OK(scanner.SetProjectedColumns({ "key" }));
     LOG_TIMING(INFO, "Scanning disk with no predicates") {
@@ -714,8 +714,8 @@ TEST_F(ClientTest, TestScanAtSnapshot) {
   ASSERT_NO_FATAL_FAILURE(InsertTestRows(client_table_.get(),
                                          half_the_rows));
 
-  // Get the time from the server and transform to micros, disregarding any
-  // logical values (we shouldn't have any with a single server anyway).
+  // get the time from the server and transform to micros disregarding any
+  // logical values (we shouldn't have any with a single server anyway);
   int64_t ts = server::HybridClock::GetPhysicalValueMicros(
       cluster_->mini_tablet_server(0)->server()->clock()->Now());
 
@@ -1274,7 +1274,7 @@ TEST_F(ClientTest, TestNonCoveringRangePartitions) {
 
   // Scans
 
-  { // Full table scan
+  { // full table scan
     vector<string> rows;
     KuduScanner scanner(table.get());
     ASSERT_OK(scanner.SetFaultTolerant());
@@ -2950,7 +2950,7 @@ TEST_F(ClientTest, TestMutateDeletedRow) {
   Status s = session->Flush();
   ASSERT_FALSE(s.ok());
   ASSERT_STR_CONTAINS(s.ToString(), "Some errors occurred");
-  // Verify error
+  // verify error
   gscoped_ptr<KuduError> error = GetSingleErrorFromSession(session.get());
   ASSERT_EQ(error->failed_op().ToString(),
             "UPDATE int32 key=1, int32 int_val=2");
@@ -2962,7 +2962,7 @@ TEST_F(ClientTest, TestMutateDeletedRow) {
   s = session->Flush();
   ASSERT_FALSE(s.ok());
   ASSERT_STR_CONTAINS(s.ToString(), "Some errors occurred");
-  // Verify error
+  // verify error
   error = GetSingleErrorFromSession(session.get());
   ASSERT_EQ(error->failed_op().ToString(),
             "DELETE int32 key=1");
@@ -2980,7 +2980,7 @@ TEST_F(ClientTest, TestMutateNonexistentRow) {
   Status s = session->Flush();
   ASSERT_FALSE(s.ok());
   ASSERT_STR_CONTAINS(s.ToString(), "Some errors occurred");
-  // Verify error
+  // verify error
   gscoped_ptr<KuduError> error = GetSingleErrorFromSession(session.get());
   ASSERT_EQ(error->failed_op().ToString(),
             "UPDATE int32 key=1, int32 int_val=2");
@@ -2992,7 +2992,7 @@ TEST_F(ClientTest, TestMutateNonexistentRow) {
   s = session->Flush();
   ASSERT_FALSE(s.ok());
   ASSERT_STR_CONTAINS(s.ToString(), "Some errors occurred");
-  // Verify error
+  // verify error
   error = GetSingleErrorFromSession(session.get());
   ASSERT_EQ(error->failed_op().ToString(),
             "DELETE int32 key=1");
@@ -3118,39 +3118,7 @@ TEST_F(ClientTest, TestWriteWithBadSchema) {
 TEST_F(ClientTest, TestBasicAlterOperations) {
   const vector<string> kBadNames = {"", string(1000, 'x')};
 
-  // Test that renaming a column to an invalid name throws an error.
-  for (const string& bad_name : kBadNames) {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("int_val")->RenameTo(bad_name);
-    Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "invalid column name");
-  }
-
-  // Test that renaming a table to an invalid name throws an error.
-  for (const string& bad_name : kBadNames) {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->RenameTo(bad_name);
-    Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "invalid table name");
-  }
-
-  // Test trying to add columns to a table such that it exceeds the permitted
-  // maximum.
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    for (int i = 0; i < 1000; i++) {
-    table_alterer->AddColumn(Substitute("c$0", i))->Type(KuduColumnSchema::INT32);
-    }
-    Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
-    ASSERT_STR_CONTAINS(s.ToString(),
-    "number of columns 1004 is greater than the "
-    "permitted maximum 300");
-  }
-
-  // Having no steps should throw an error.
+  // test that having no steps throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     Status s = table_alterer->Alter();
@@ -3158,7 +3126,7 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_STR_CONTAINS(s.ToString(), "No alter steps provided");
   }
 
-  // Adding a non-nullable column with no default value should throw an error.
+  // test that adding a non-nullable column with no default value throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     table_alterer->AddColumn("key")->Type(KuduColumnSchema::INT32)->NotNull();
@@ -3167,26 +3135,26 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_STR_CONTAINS(s.ToString(), "column `key`: NOT NULL columns must have a default");
   }
 
-  // Removing a key should throw an error.
+  // test that remove key should throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     Status s = table_alterer
       ->DropColumn("key")
       ->Alter();
     ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "cannot remove a key column: key");
+    ASSERT_STR_CONTAINS(s.ToString(), "cannot remove a key column");
   }
 
-  // Renaming a key should throw an error.
+  // test that renaming a key should throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     table_alterer->AlterColumn("key")->RenameTo("key2");
     Status s = table_alterer->Alter();
     ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "cannot alter a key column: key");
+    ASSERT_STR_CONTAINS(s.ToString(), "cannot rename a key column");
   }
 
-  // Renaming a column to an already-existing name should throw an error.
+  // test that renaming to an already-existing name throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     table_alterer->AlterColumn("int_val")->RenameTo("string_val");
@@ -3195,38 +3163,36 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_STR_CONTAINS(s.ToString(), "The column already exists: string_val");
   }
 
-  // Altering a column but specifying no alterations should throw an error.
-  {
+  // Test that renaming a column to an invalid name throws an error.
+  for (const string& bad_name : kBadNames) {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val");
+    table_alterer->AlterColumn("int_val")->RenameTo(bad_name);
     Status s = table_alterer->Alter();
     ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "no alter operation specified: string_val");
-  }
-
-  // Trying to change type or nullability of a column is an error.
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")->Type(KuduColumnSchema::STRING);
-    Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsNotSupported());
-    ASSERT_STR_CONTAINS(s.ToString(), "unsupported alter operation: string_val");
+    ASSERT_STR_CONTAINS(s.ToString(), "invalid column name");
   }
 
-  {
+  // Test that renaming a table to an invalid name throws an error.
+  for (const string& bad_name : kBadNames) {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")->Nullable();
+    table_alterer->RenameTo(bad_name);
     Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsNotSupported());
-    ASSERT_STR_CONTAINS(s.ToString(), "unsupported alter operation: string_val");
+    ASSERT_TRUE(s.IsInvalidArgument());
+    ASSERT_STR_CONTAINS(s.ToString(), "invalid table name");
   }
 
+  // Test trying to add columns to a table such that it exceeds the permitted
+  // maximum.
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")->NotNull();
+    for (int i = 0; i < 1000; i++) {
+      table_alterer->AddColumn(Substitute("c$0", i))->Type(KuduColumnSchema::INT32);
+    }
     Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsNotSupported());
-    ASSERT_STR_CONTAINS(s.ToString(), "unsupported alter operation: string_val");
+    ASSERT_TRUE(s.IsInvalidArgument()) << s.ToString();
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "number of columns 1004 is greater than the "
+                        "permitted maximum 300");
   }
 
   // Need a tablet peer for the next set of tests.
@@ -3243,7 +3209,8 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_EQ(1, tablet_peer->tablet()->metadata()->schema_version());
   }
 
-  // Specifying an encoding incompatible with the column's type is an error.
+  // test that specifying an encoding incompatible with the column's
+  // type throws an error
   {
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     table_alterer->AddColumn("new_string_val")->Type(KuduColumnSchema::STRING)
@@ -3262,103 +3229,13 @@ TEST_F(ClientTest, TestBasicAlterOperations) {
     ASSERT_EQ(2, tablet_peer->tablet()->metadata()->schema_version());
   }
 
-  // Test changing a default value for a binary column.
-  // There are separate tests for fixed- and variable-length types because
-  // the implementations differ a little
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")
-        ->Default(KuduValue::CopyString("hello!"));
-    ASSERT_OK(table_alterer->Alter());
-    ASSERT_EQ(3, tablet_peer->tablet()->metadata()->schema_version());
-    Schema schema = tablet_peer->tablet()->metadata()->schema();
-    ColumnSchema col_schema = schema.column(schema.find_column("string_val"));
-    ASSERT_FALSE(col_schema.has_read_default());
-    ASSERT_TRUE(col_schema.has_write_default());
-    ASSERT_EQ("hello!", *reinterpret_cast<const Slice*>(col_schema.write_default_value()));
-  }
-
-  // Change a default value for an integer column.
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("non_null_with_default")
-        ->Default(KuduValue::FromInt(54321));
-    ASSERT_OK(table_alterer->Alter());
-    ASSERT_EQ(4, tablet_peer->tablet()->metadata()->schema_version());
-    Schema schema = tablet_peer->tablet()->metadata()->schema();
-    ColumnSchema col_schema = schema.column(schema.find_column("non_null_with_default"));
-    ASSERT_TRUE(col_schema.has_read_default()); // Started with a default
-    ASSERT_TRUE(col_schema.has_write_default());
-    ASSERT_EQ(54321, *reinterpret_cast<const int32_t*>(col_schema.write_default_value()));
-  }
-
-  // Clear a default value from a nullable column
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")
-        ->RemoveDefault();
-    ASSERT_OK(table_alterer->Alter());
-    ASSERT_EQ(5, tablet_peer->tablet()->metadata()->schema_version());
-    Schema schema = tablet_peer->tablet()->metadata()->schema();
-    ColumnSchema col_schema = schema.column(schema.find_column("string_val"));
-    ASSERT_FALSE(col_schema.has_read_default());
-    ASSERT_FALSE(col_schema.has_write_default());
-  }
-
-  // Clear a default value from a non-nullable column
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("non_null_with_default")
-        ->RemoveDefault();
-    ASSERT_OK(table_alterer->Alter());
-    ASSERT_EQ(6, tablet_peer->tablet()->metadata()->schema_version());
-    Schema schema = tablet_peer->tablet()->metadata()->schema();
-    ColumnSchema col_schema = schema.column(schema.find_column("non_null_with_default"));
-    ASSERT_TRUE(col_schema.has_read_default());
-    ASSERT_FALSE(col_schema.has_write_default());
-  }
-
-  // Test that specifying a default of the wrong size fails.
-  // Note that, since the column's type isn't checked (or necessarily known)
-  // client-side, the server just receives some bytes, and, since the client
-  // stores 64-bit integer values as defaults for all integer-backed types, the
-  // value might be too large. We'll still be ok if we just truncate, though.
-  // This is actually how it works for regular add column and create table, but
-  // there the type of the column is known, so the truncation is backed up by
-  // column type-checking, and it can't be the case that there's too few bytes.
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("non_null_with_default")
-        ->Default(KuduValue::CopyString("aaa"));
-    Status s = table_alterer->Alter();
-    ASSERT_TRUE(s.IsInvalidArgument());
-    ASSERT_STR_CONTAINS(s.ToString(), "wrong size for default value");
-    ASSERT_EQ(6, tablet_peer->tablet()->metadata()->schema_version());
-  }
-
-  // Test altering encoding, compression, and block size.
-  {
-    gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("string_val")
-        ->Encoding(KuduColumnStorageAttributes::DICT_ENCODING)
-        ->Compression(KuduColumnStorageAttributes::LZ4)
-        ->BlockSize(16 * 1024 * 1024);
-    ASSERT_OK(table_alterer->Alter());
-    ASSERT_EQ(7, tablet_peer->tablet()->metadata()->schema_version());
-    Schema schema = tablet_peer->tablet()->metadata()->schema();
-    ColumnSchema col_schema = schema.column(schema.find_column("string_val"));
-    ASSERT_EQ(KuduColumnStorageAttributes::DICT_ENCODING, col_schema.attributes().encoding);
-    ASSERT_EQ(KuduColumnStorageAttributes::LZ4, col_schema.attributes().compression);
-    ASSERT_EQ(16 * 1024 * 1024, col_schema.attributes().cfile_block_size);
-  }
-
   {
     const char *kRenamedTableName = "RenamedTable";
     gscoped_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
     ASSERT_OK(table_alterer
               ->RenameTo(kRenamedTableName)
               ->Alter());
-    ASSERT_EQ(8, tablet_peer->tablet()->metadata()->schema_version());
+    ASSERT_EQ(3, tablet_peer->tablet()->metadata()->schema_version());
     ASSERT_EQ(kRenamedTableName, tablet_peer->tablet()->metadata()->table_name());
 
     CatalogManager *catalog_manager = cluster_->mini_master()->master()->catalog_manager();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index 8009ac2..516dfcd 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -17,7 +17,6 @@
 
 #include "kudu/client/schema.h"
 
-#include <boost/optional.hpp>
 #include <glog/logging.h>
 #include <unordered_map>
 
@@ -205,6 +204,8 @@ Status KuduColumnSpec::ToColumnSchema(KuduColumnSchema* col) const {
   // Verify that the user isn't trying to use any methods that
   // don't make sense for CREATE.
   if (data_->has_rename_to) {
+    // TODO(KUDU-861): adjust these errors as this method will also be used for
+    // ALTER TABLE ADD COLUMN support.
     return Status::NotSupported("cannot rename a column during CreateTable",
                                 data_->name);
   }
@@ -253,57 +254,6 @@ Status KuduColumnSpec::ToColumnSchema(KuduColumnSchema* col) const {
   return Status::OK();
 }
 
-Status KuduColumnSpec::ToColumnSchemaDelta(ColumnSchemaDelta* col_delta) const {
-  if (data_->has_type) {
-    return Status::InvalidArgument("type provided for column schema delta", data_->name);
-  }
-  if (data_->has_nullable) {
-    return Status::InvalidArgument("nullability provided for column schema delta", data_->name);
-  }
-  if (data_->primary_key) {
-    return Status::InvalidArgument("primary key set for column schema delta", data_->name);
-  }
-
-  if (data_->has_rename_to) {
-    col_delta->new_name = boost::optional<string>(std::move(data_->rename_to));
-  }
-
-  if (data_->has_default) {
-    col_delta->default_value = boost::optional<Slice>(DefaultValueAsSlice());
-  }
-
-  if (data_->remove_default) {
-    col_delta->remove_default = true;
-  }
-
-  if (col_delta->remove_default && col_delta->default_value) {
-    return Status::InvalidArgument("new default set but default also removed", data_->name);
-  }
-
-  if (data_->has_encoding) {
-    col_delta->encoding =
-            boost::optional<EncodingType>(ToInternalEncodingType(data_->encoding));
-  }
-
-  if (data_->has_compression) {
-    col_delta->compression =
-            boost::optional<CompressionType>(ToInternalCompressionType(data_->compression));
-  }
-
-  if (data_->has_block_size) {
-    col_delta->cfile_block_size = boost::optional<int32_t>(data_->block_size);
-  }
-
-  return Status::OK();
-}
-
-Slice KuduColumnSpec::DefaultValueAsSlice() const {
-  if (!data_->has_default) {
-    return Slice();
-  }
-  return data_->default_val->data_->GetSlice();
-}
-
 
 ////////////////////////////////////////////////////////////
 // KuduSchemaBuilder

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/schema.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index 04eb553..5bdba4d 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -27,7 +27,6 @@
 namespace kudu {
 
 class ColumnSchema;
-struct ColumnSchemaDelta;
 class KuduPartialRow;
 class Schema;
 class TestWorkload;
@@ -350,10 +349,6 @@ class KUDU_EXPORT KuduColumnSpec {
 
   Status ToColumnSchema(KuduColumnSchema* col) const;
 
-  Status ToColumnSchemaDelta(ColumnSchemaDelta* col_delta) const;
-
-  Slice DefaultValueAsSlice() const;
-
   // Owned.
   Data* data_;
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/table_alterer-internal.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/table_alterer-internal.cc b/src/kudu/client/table_alterer-internal.cc
index ba86b55..3ea2a16 100644
--- a/src/kudu/client/table_alterer-internal.cc
+++ b/src/kudu/client/table_alterer-internal.cc
@@ -86,31 +86,29 @@ Status KuduTableAlterer::Data::ToRequest(AlterTableRequestPB* req) {
         break;
       }
       case AlterTableRequestPB::ALTER_COLUMN:
-      {
-        // We only support rename column, change block_size, encoding, and compression
+        // TODO(KUDU-861): support altering a column in the wire protocol.
+        // For now, we just give an error if the caller tries to do
+        // any operation other than rename.
         if (s.spec->data_->has_type ||
+            s.spec->data_->has_encoding ||
+            s.spec->data_->has_compression ||
             s.spec->data_->has_nullable ||
-            s.spec->data_->primary_key) {
-          return Status::NotSupported("unsupported alter operation",
+            s.spec->data_->primary_key ||
+            s.spec->data_->has_default ||
+            s.spec->data_->default_val ||
+            s.spec->data_->remove_default) {
+          return Status::NotSupported("cannot support AlterColumn of this type",
                                       s.spec->data_->name);
         }
-        if (!s.spec->data_->has_rename_to &&
-            !s.spec->data_->has_default &&
-            !s.spec->data_->default_val &&
-            !s.spec->data_->remove_default &&
-            !s.spec->data_->has_encoding &&
-            !s.spec->data_->has_compression &&
-            !s.spec->data_->has_block_size) {
+        // We only support rename column
+        if (!s.spec->data_->has_rename_to) {
           return Status::InvalidArgument("no alter operation specified",
                                          s.spec->data_->name);
         }
-        ColumnSchemaDelta col_delta(s.spec->data_->name);
-        RETURN_NOT_OK(s.spec->ToColumnSchemaDelta(&col_delta));
-        auto* alter_pb = pb_step->mutable_alter_column();
-        ColumnSchemaDeltaToPB(col_delta, alter_pb->mutable_delta());
-        pb_step->set_type(AlterTableRequestPB::ALTER_COLUMN);
+        pb_step->mutable_rename_column()->set_old_name(s.spec->data_->name);
+        pb_step->mutable_rename_column()->set_new_name(s.spec->data_->rename_to);
+        pb_step->set_type(AlterTableRequestPB::RENAME_COLUMN);
         break;
-      }
       case AlterTableRequestPB::ADD_RANGE_PARTITION:
       {
         RowOperationsPBEncoder encoder(pb_step->mutable_add_range_partition()

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/value-internal.h
----------------------------------------------------------------------
diff --git a/src/kudu/client/value-internal.h b/src/kudu/client/value-internal.h
index 590a75c..c66a7bc 100644
--- a/src/kudu/client/value-internal.h
+++ b/src/kudu/client/value-internal.h
@@ -55,10 +55,6 @@ class KuduValue::Data {
                                 DataType t,
                                 void** val_void);
 
-  // Return this KuduValue as a Slice. The KuduValue object retains ownership
-  // of the underlying data.
-  const Slice GetSlice();
-
  private:
   // Check that this value has the expected type 'type', returning
   // a nice error Status if not.

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/client/value.cc
----------------------------------------------------------------------
diff --git a/src/kudu/client/value.cc b/src/kudu/client/value.cc
index 2034b46..7971748 100644
--- a/src/kudu/client/value.cc
+++ b/src/kudu/client/value.cc
@@ -188,23 +188,5 @@ Status KuduValue::Data::CheckAndPointToString(const string& col_name,
   return Status::OK();
 }
 
-const Slice KuduValue::Data::GetSlice() {
-  switch (type_) {
-    case INT: {
-      return Slice(reinterpret_cast<uint8_t *>(&int_val_),
-                   sizeof(int64_t));
-    }
-    case FLOAT:
-      return Slice(reinterpret_cast<uint8_t*>(&float_val_),
-                   sizeof(float));
-    case DOUBLE:
-      return Slice(reinterpret_cast<uint8_t*>(&double_val_),
-                   sizeof(double));
-    case SLICE:
-      return slice_val_;
-  }
-  LOG(FATAL);
-}
-
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/common/common.proto
----------------------------------------------------------------------
diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index c5eced0..c85e65b 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -101,18 +101,6 @@ message ColumnSchemaPB {
   optional int32 cfile_block_size = 10 [default=0];
 }
 
-message ColumnSchemaDeltaPB {
-  optional string name = 1;
-  optional string new_name = 2;
-
-  optional bytes default_value = 4;
-  optional bool remove_default = 5;
-
-  optional EncodingType encoding = 6;
-  optional CompressionType compression = 7;
-  optional int32 block_size = 8;
-}
-
 message SchemaPB {
   repeated ColumnSchemaPB columns = 1;
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/common/schema.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index 2d44670..bd83088 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -17,10 +17,9 @@
 
 #include "kudu/common/schema.h"
 
-#include <algorithm>
 #include <set>
+#include <algorithm>
 
-#include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/join.h"
 #include "kudu/gutil/strings/strcat.h"
@@ -30,7 +29,8 @@
 
 namespace kudu {
 
-using std::shared_ptr;
+using std::set;
+using std::unordered_map;
 using std::unordered_set;
 
 // In a new schema, we typically would start assigning column IDs at 0. However, this
@@ -52,44 +52,7 @@ string ColumnStorageAttributes::ToString() const {
                              cfile_block_size);
 }
 
-Status ColumnSchema::ApplyDelta(const ColumnSchemaDelta& col_delta) {
-  // This method does all validation up-front before making any changes to
-  // the schema, so that if we return an error then we are guaranteed to
-  // have had no effect
-  if (type_info()->physical_type() != BINARY) {
-    if (col_delta.default_value && col_delta.default_value->size() < type_info()->size()) {
-      return Status::InvalidArgument("wrong size for default value");
-    }
-  }
-
-  if (col_delta.new_name) {
-    name_ = *col_delta.new_name;
-  }
-
-  if (col_delta.default_value) {
-    const void* value = type_info()->physical_type() == BINARY ?
-                        reinterpret_cast<const void *>(&col_delta.default_value.get()) :
-                        reinterpret_cast<const void *>(col_delta.default_value->data());
-    write_default_ = std::make_shared<Variant>(type_info()->type(), value);
-  }
-
-  if (col_delta.remove_default) {
-    write_default_ = nullptr;
-  }
-
-  if (col_delta.encoding) {
-    attributes_.encoding = *col_delta.encoding;
-  }
-  if (col_delta.compression) {
-    attributes_.compression = *col_delta.compression;
-  }
-  if (col_delta.cfile_block_size) {
-    attributes_.cfile_block_size = *col_delta.cfile_block_size;
-  }
-  return Status::OK();
-}
-
-// TODO(wdb): include attributes_.ToString() -- need to fix unit tests
+// TODO: include attributes_.ToString() -- need to fix unit tests
 // first
 string ColumnSchema::ToString() const {
   return strings::Substitute("$0[$1]",
@@ -454,8 +417,8 @@ Status SchemaBuilder::AddColumn(const string& name,
 }
 
 Status SchemaBuilder::RemoveColumn(const string& name) {
-  unordered_set<string>::const_iterator it_names = col_names_.find(name);
-  if (it_names == col_names_.end()) {
+  unordered_set<string>::const_iterator it_names;
+  if ((it_names = col_names_.find(name)) == col_names_.end()) {
     return Status::NotFound("The specified column does not exist", name);
   }
 
@@ -476,18 +439,19 @@ Status SchemaBuilder::RemoveColumn(const string& name) {
 }
 
 Status SchemaBuilder::RenameColumn(const string& old_name, const string& new_name) {
+  unordered_set<string>::const_iterator it_names;
+
   // check if 'new_name' is already in use
-  if (col_names_.find(new_name) != col_names_.end()) {
+  if ((it_names = col_names_.find(new_name)) != col_names_.end()) {
     return Status::AlreadyPresent("The column already exists", new_name);
   }
 
   // check if the 'old_name' column exists
-  unordered_set<string>::const_iterator it_names = col_names_.find(old_name);
-  if (it_names == col_names_.end()) {
+  if ((it_names = col_names_.find(old_name)) == col_names_.end()) {
     return Status::NotFound("The specified column does not exist", old_name);
   }
 
-  col_names_.erase(it_names);   // TODO(wdb): Should this one stay and marked as alias?
+  col_names_.erase(it_names);   // TODO: Should this one stay and marked as alias?
   col_names_.insert(new_name);
 
   for (ColumnSchema& col_schema : cols_) {
@@ -520,31 +484,4 @@ Status SchemaBuilder::AddColumn(const ColumnSchema& column, bool is_key) {
   return Status::OK();
 }
 
-Status SchemaBuilder::ApplyColumnSchemaDelta(const ColumnSchemaDelta& col_delta) {
-  // if the column will be renamed, check if 'new_name' is already in use
-  if (col_delta.new_name && ContainsKey(col_names_, *col_delta.new_name)) {
-    return Status::AlreadyPresent("The column already exists", *col_delta.new_name);
-  }
-
-  // check if the column exists
-  unordered_set<string>::const_iterator it_names = col_names_.find(col_delta.name);
-  if (it_names == col_names_.end()) {
-    return Status::NotFound("The specified column does not exist", col_delta.name);
-  }
-
-  for (ColumnSchema& col_schema : cols_) {
-    if (col_delta.name == col_schema.name()) {
-      RETURN_NOT_OK(col_schema.ApplyDelta(col_delta));
-      if (col_delta.new_name) {
-        // TODO(wdberkeley): Should the old one stay, marked as an alias?
-        col_names_.erase(it_names);
-        col_names_.insert(*col_delta.new_name);
-      }
-      return Status::OK();
-    }
-  }
-
-  LOG(FATAL) << "Should not reach here";
-}
-
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/common/schema.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index bdf3ccf..b8656e9 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -17,7 +17,6 @@
 #ifndef KUDU_COMMON_SCHEMA_H
 #define KUDU_COMMON_SCHEMA_H
 
-#include <boost/optional.hpp>
 #include <functional>
 #include <glog/logging.h>
 #include <memory>
@@ -105,37 +104,6 @@ struct ColumnStorageAttributes {
   int32_t cfile_block_size;
 };
 
-// A struct representing changes to a ColumnSchema.
-//
-// In the future, as more complex alter operations need to be supported,
-// this may evolve into a class, but for now it's just POD.
-struct ColumnSchemaDelta {
-public:
-  explicit ColumnSchemaDelta(std::string name)
-      : name(std::move(name)),
-        remove_default(false) {
-  }
-
-  const std::string name;
-
-  boost::optional<std::string> new_name;
-
-  // NB: these properties of a column cannot be changed yet,
-  // ergo type and nullable should always be empty.
-  // TODO(wdberkeley) allow changing of type and nullability
-  boost::optional<DataType> type;
-
-  boost::optional<bool> nullable;
-
-  boost::optional<Slice> default_value;
-
-  bool remove_default;
-
-  boost::optional<EncodingType> encoding;
-  boost::optional<CompressionType> compression;
-  boost::optional<int32_t> cfile_block_size;
-};
-
 // The schema for a given column.
 //
 // Holds the data type as well as information about nullability & column name.
@@ -170,6 +138,7 @@ class ColumnSchema {
     if (write_default == read_default) {
       write_default_ = read_default_;
     } else if (write_default != NULL) {
+      DCHECK(read_default != NULL) << "Must have a read default";
       write_default_.reset(new Variant(type, write_default));
     }
   }
@@ -265,11 +234,6 @@ class ColumnSchema {
     return true;
   }
 
-  // Apply a ColumnSchemaDelta to this column schema, altering it according to
-  // the changes in the delta.
-  // The original column schema is not changed unless ApplyDelta returns OK.
-  Status ApplyDelta(const ColumnSchemaDelta& col_delta);
-
   // Returns extended attributes (such as encoding, compression, etc...)
   // associated with the column schema. The reason they are kept in a separate
   // struct is so that in the future, they may be moved out to a more
@@ -884,11 +848,8 @@ class SchemaBuilder {
                    const void *write_default);
 
   Status RemoveColumn(const string& name);
-
   Status RenameColumn(const string& old_name, const string& new_name);
 
-  Status ApplyColumnSchemaDelta(const ColumnSchemaDelta& col_delta);
-
  private:
   DISALLOW_COPY_AND_ASSIGN(SchemaBuilder);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/common/wire_protocol.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index f8cf561..0c3d23b 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -17,7 +17,6 @@
 
 #include "kudu/common/wire_protocol.h"
 
-#include <boost/optional.hpp>
 #include <string>
 #include <vector>
 
@@ -256,53 +255,6 @@ ColumnSchema ColumnSchemaFromPB(const ColumnSchemaPB& pb) {
                       attributes);
 }
 
-void ColumnSchemaDeltaToPB(const ColumnSchemaDelta& col_delta, ColumnSchemaDeltaPB *pb) {
-  pb->Clear();
-  pb->set_name(col_delta.name);
-  if (col_delta.new_name) {
-    pb->set_new_name(*col_delta.new_name);
-  }
-  if (col_delta.default_value) {
-    pb->set_default_value(col_delta.default_value->data(),
-                          col_delta.default_value->size());
-  }
-  if (col_delta.remove_default) {
-    pb->set_remove_default(true);
-  }
-  if (col_delta.encoding) {
-    pb->set_encoding(*col_delta.encoding);
-  }
-  if (col_delta.compression) {
-    pb->set_compression(*col_delta.compression);
-  }
-  if (col_delta.cfile_block_size) {
-    pb->set_block_size(*col_delta.cfile_block_size);
-  }
-}
-
-ColumnSchemaDelta ColumnSchemaDeltaFromPB(const ColumnSchemaDeltaPB& pb) {
-  ColumnSchemaDelta col_delta(pb.name());
-  if (pb.has_new_name()) {
-    col_delta.new_name = boost::optional<string>(pb.new_name());
-  }
-  if (pb.has_default_value()) {
-    col_delta.default_value = boost::optional<Slice>(Slice(pb.default_value()));
-  }
-  if (pb.has_remove_default()) {
-    col_delta.remove_default = true;
-  }
-  if (pb.has_encoding()) {
-    col_delta.encoding = boost::optional<EncodingType>(pb.encoding());
-  }
-  if (pb.has_compression()) {
-    col_delta.compression = boost::optional<CompressionType>(pb.compression());
-  }
-  if (pb.has_block_size()) {
-    col_delta.cfile_block_size = boost::optional<int32_t>(pb.block_size());
-  }
-  return col_delta;
-}
-
 Status ColumnPBsToSchema(const RepeatedPtrField<ColumnSchemaPB>& column_pbs,
                          Schema* schema) {
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/common/wire_protocol.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol.h b/src/kudu/common/wire_protocol.h
index d2743b7..6448282 100644
--- a/src/kudu/common/wire_protocol.h
+++ b/src/kudu/common/wire_protocol.h
@@ -32,7 +32,6 @@ namespace kudu {
 class Arena;
 class ColumnPredicate;
 class ColumnSchema;
-struct ColumnSchemaDelta;
 class ConstContiguousRow;
 class faststring;
 class HostPort;
@@ -86,13 +85,6 @@ void ColumnSchemaToPB(const ColumnSchema& schema, ColumnSchemaPB *pb, int flags
 // Return the ColumnSchema created from the specified protobuf.
 ColumnSchema ColumnSchemaFromPB(const ColumnSchemaPB& pb);
 
-// Convert the column schema delta `col_delta` to protobuf.
-void ColumnSchemaDeltaToPB(const ColumnSchemaDelta& col_delta, ColumnSchemaDeltaPB *pb);
-
-// Return the ColumnSchemaDelta created from the protobuf `pb`.
-// The protobuf must outlive the returned ColumnSchemaDelta.
-ColumnSchemaDelta ColumnSchemaDeltaFromPB(const ColumnSchemaDeltaPB& pb);
-
 // Convert the given list of ColumnSchemaPB objects into a Schema object.
 //
 // Returns InvalidArgument if the provided columns don't make a valid Schema

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/integration-tests/alter_table-randomized-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-randomized-test.cc b/src/kudu/integration-tests/alter_table-randomized-test.cc
index ab7cd5a..55b5318 100644
--- a/src/kudu/integration-tests/alter_table-randomized-test.cc
+++ b/src/kudu/integration-tests/alter_table-randomized-test.cc
@@ -36,9 +36,10 @@
 namespace kudu {
 
 using client::KuduClient;
+using client::KuduClientBuilder;
 using client::KuduColumnSchema;
-using client::KuduColumnStorageAttributes;
 using client::KuduError;
+using client::KuduInsert;
 using client::KuduScanner;
 using client::KuduSchema;
 using client::KuduSchemaBuilder;
@@ -60,18 +61,6 @@ using strings::SubstituteAndAppend;
 const char* kTableName = "test-table";
 const int kMaxColumns = 30;
 const uint32_t kMaxRangePartitions = 32;
-const vector<KuduColumnStorageAttributes::CompressionType> kCompressionTypes =
-    { KuduColumnStorageAttributes::NO_COMPRESSION,
-      KuduColumnStorageAttributes::SNAPPY,
-      KuduColumnStorageAttributes::LZ4,
-      KuduColumnStorageAttributes::ZLIB };
-const vector <KuduColumnStorageAttributes::EncodingType> kInt32Encodings =
-      { KuduColumnStorageAttributes::PLAIN_ENCODING,
-        KuduColumnStorageAttributes::RLE,
-        KuduColumnStorageAttributes::BIT_SHUFFLE };
-// A block size of 0 applies the server-side default.
-const vector<int32_t> kBlockSizes = {0, 2 * 1024 * 1024,
-                                     4 * 1024 * 1024, 8 * 1024 * 1024};
 
 class AlterTableRandomized : public KuduTest {
  public:
@@ -126,10 +115,6 @@ struct RowState {
   // We ensure that we never insert or update to this value except in the case of
   // NULLable columns.
   static const int32_t kNullValue = 0xdeadbeef;
-  // We use this special value to denote default values.
-  // We ensure that we never insert or update to this value except when the
-  // column should be assigned its current default value.
-  static const int32_t kDefaultValue = 0xbabecafe;
   vector<pair<string, int32_t>> cols;
 
   string ToString() const {
@@ -156,7 +141,6 @@ struct TableState {
       : rand_(SeedRandom()) {
     col_names_.push_back("key");
     col_nullable_.push_back(false);
-    col_defaults_.push_back(0);
     AddRangePartition();
   }
 
@@ -201,7 +185,7 @@ struct TableState {
     int32_t key = GetRandomNewRowKey();
 
     int32_t seed = rand_.Next();
-    if (seed == RowState::kNullValue || seed == RowState::kDefaultValue) {
+    if (seed == RowState::kNullValue) {
       seed++;
     }
 
@@ -211,8 +195,6 @@ struct TableState {
       int32_t val;
       if (col_nullable_[i] && seed % 2 == 1) {
         val = RowState::kNullValue;
-      } else if (seed % 3 == 0) {
-        val = RowState::kDefaultValue;
       } else {
         val = seed;
       }
@@ -227,11 +209,6 @@ struct TableState {
 
     auto r = new RowState;
     r->cols = data;
-    for (int i = 1; i < r->cols.size(); i++) {
-      if (r->cols[i].second == RowState::kDefaultValue) {
-        r->cols[i].second = col_defaults_[i];
-      }
-    }
     rows_[key].reset(r);
     return true;
   }
@@ -253,7 +230,6 @@ struct TableState {
   void AddColumnWithDefault(const string& name, int32_t def, bool nullable) {
     col_names_.push_back(name);
     col_nullable_.push_back(nullable);
-    col_defaults_.push_back(def);
     for (auto& e : rows_) {
       e.second->cols.push_back(make_pair(name, def));
     }
@@ -264,30 +240,15 @@ struct TableState {
     int index = col_it - col_names_.begin();
     col_names_.erase(col_it);
     col_nullable_.erase(col_nullable_.begin() + index);
-    col_defaults_.erase(col_defaults_.begin() + index);
     for (auto& e : rows_) {
       e.second->cols.erase(e.second->cols.begin() + index);
     }
   }
 
-  void RenameColumn(const string& existing_name, const string& new_name) {
+  void RenameColumn(const string& existing_name, string new_name) {
     auto iter = std::find(col_names_.begin(), col_names_.end(), existing_name);
     CHECK(iter != col_names_.end());
-    *iter = new_name;
-
-    for (auto& e : rows_) {
-      for (auto& pair : e.second->cols) {
-        if (pair.first == existing_name) {
-          pair.first = new_name;
-        }
-      }
-    }
-  }
-
-  void ChangeDefault(const string& name, int32_t new_def) {
-    auto col_it = std::find(col_names_.begin(), col_names_.end(), name);
-    CHECK(col_it != col_names_.end());
-    col_defaults_[col_it - col_names_.begin()] = new_def;
+    *iter = std::move(new_name);
   }
 
   pair<int32_t, int32_t> AddRangePartition() {
@@ -333,10 +294,6 @@ struct TableState {
   // Has the same length as col_names_.
   vector<bool> col_nullable_;
 
-  // For each column, its current write default.
-  // Has the same length as col_names_.
-  vector<int32_t> col_defaults_;
-
   map<int32_t, unique_ptr<RowState>> rows_;
 
   // The lower and upper bounds of all range partitions in the table.
@@ -402,7 +359,6 @@ struct MirrorTable {
     for (int i = 1; i < num_columns(); i++) {
       int32_t val = rand * i;
       if (val == RowState::kNullValue) val++;
-      if (val == RowState::kDefaultValue) val++;
       if (ts_.col_nullable_[i] && val % 2 == 1) {
         val = RowState::kNullValue;
       }
@@ -435,19 +391,13 @@ struct MirrorTable {
 
     int step_count = 1 + ts_.rand_.Uniform(10);
     for (int step = 0; step < step_count; step++) {
-      int r = ts_.rand_.Uniform(7);
+      int r = ts_.rand_.Uniform(4);
       if (r < 1 && num_columns() < kMaxColumns) {
         AddAColumn(table_alterer.get());
       } else if (r < 2 && num_columns() > 1) {
         DropAColumn(table_alterer.get());
-      } else if (r < 3 && num_columns() > 1) {
-        RenameAColumn(table_alterer.get());
-      } else if (r < 4 && num_columns() > 1) {
-        ChangeADefault(table_alterer.get());
-      } else if (r < 5 && num_columns() > 1) {
-        ChangeAStorageAttribute(table_alterer.get());
       } else if (num_range_partitions() == 0 ||
-                 (r < 6 && num_range_partitions() < kMaxRangePartitions)) {
+                 (r < 3 && num_range_partitions() < kMaxRangePartitions)) {
         AddARangePartition(schema, table_alterer.get());
       } else {
         DropARangePartition(schema, table_alterer.get());
@@ -491,42 +441,10 @@ struct MirrorTable {
     string new_name = ts_.GetRandomNewColumnName();
     LOG(INFO) << "Renaming column " << original_name << " to " << new_name;
     table_alterer->AlterColumn(original_name)->RenameTo(new_name);
-    ts_.RenameColumn(original_name, new_name);
-  }
-
-  void ChangeADefault(KuduTableAlterer* table_alterer) {
-    string name = ts_.GetRandomExistingColumnName();
-    int32_t new_def = ts_.rand_.Next();
-    if (new_def == RowState::kNullValue || new_def == RowState::kDefaultValue) {
-      new_def++;
-    }
-    LOG(INFO) << "Changing default of column " << name << " to " << new_def;
-    table_alterer->AlterColumn(name)->Default(KuduValue::FromInt(new_def));
-    ts_.ChangeDefault(name, new_def);
-  }
-
-  void ChangeAStorageAttribute(KuduTableAlterer* table_alterer) {
-    string name = ts_.GetRandomExistingColumnName();
-    int type = ts_.rand_.Uniform(3);
-    if (type == 0) {
-      int i = ts_.rand_.Uniform(kCompressionTypes.size());
-      LOG(INFO) << "Changing compression of column " << name <<
-          " to " << kCompressionTypes[i];
-      table_alterer->AlterColumn(name)->Compression(kCompressionTypes[i]);
-    } else if (type == 1) {
-      int i = ts_.rand_.Uniform(kInt32Encodings.size());
-      LOG(INFO) << "Changing encoding of column " << name <<
-          " to " << kInt32Encodings[i];
-      table_alterer->AlterColumn(name)->Encoding(kInt32Encodings[i]);
-    } else {
-      int i = ts_.rand_.Uniform(kBlockSizes.size());
-      LOG(INFO) << "Changing block size of column " << name <<
-          " to " << kBlockSizes[i];
-      table_alterer->AlterColumn(name)->BlockSize(kBlockSizes[i]);
-    }
+    ts_.RenameColumn(original_name, std::move(new_name));
   }
 
-  void AddARangePartition(const KuduSchema& schema, KuduTableAlterer* table_alterer) {
+  void AddARangePartition(KuduSchema& schema, KuduTableAlterer* table_alterer) {
     auto bounds = ts_.AddRangePartition();
     LOG(INFO) << "Adding range partition: [" << bounds.first << ", " << bounds.second << ")"
               << " resulting partitions: ("
@@ -627,16 +545,9 @@ struct MirrorTable {
       case UPDATE: op.reset(table->NewUpdate()); break;
       case DELETE: op.reset(table->NewDelete()); break;
     }
-    for (int i = 0; i < data.size(); i++) {
-      const auto& d = data[i];
+    for (const auto& d : data) {
       if (d.second == RowState::kNullValue) {
         CHECK_OK(op->mutable_row()->SetNull(d.first));
-      } else if (d.second == RowState::kDefaultValue) {
-        if (ts_.col_defaults_[i] == RowState::kNullValue) {
-          CHECK_OK(op->mutable_row()->SetNull(d.first));
-        } else {
-          CHECK_OK(op->mutable_row()->SetInt32(d.first, ts_.col_defaults_[i]));
-        }
       } else {
         CHECK_OK(op->mutable_row()->SetInt32(d.first, d.second));
       }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/integration-tests/alter_table-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/alter_table-test.cc b/src/kudu/integration-tests/alter_table-test.cc
index 4b901dd..2d0b8eb 100644
--- a/src/kudu/integration-tests/alter_table-test.cc
+++ b/src/kudu/integration-tests/alter_table-test.cc
@@ -345,116 +345,6 @@ TEST_F(AlterTableTest, TestAddNullableColumnWithoutDefault) {
   EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 new=NULL)", rows[1]);
 }
 
-// Test altering a column to add a default value, change a default value, and
-// remove a default value.
-TEST_F(AlterTableTest, TestAddChangeRemoveColumnDefaultValue) {
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AddColumn("newInt32")->Type(KuduColumnSchema::INT32);
-    table_alterer->AddColumn("newString")->Type(KuduColumnSchema::STRING);
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  InsertRows(0, 1);
-  ASSERT_OK(tablet_peer_->tablet()->Flush());
-
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("newInt32")->Default(KuduValue::FromInt(12345));
-    table_alterer->AlterColumn("newString")->Default(KuduValue::CopyString("taco"));
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  InsertRows(1, 1);
-  ASSERT_OK(tablet_peer_->tablet()->Flush());
-
-  vector<string> rows;
-  ScanToStrings(&rows);
-  ASSERT_EQ(2, rows.size());
-  EXPECT_EQ("(int32 c0=0, int32 c1=0, int32 newInt32=NULL, string newString=NULL)", rows[0]);
-  EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 newInt32=12345, string newString=\"taco\")",
-            rows[1]);
-
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("newInt32")->Default(KuduValue::FromInt(54321));
-    table_alterer->AlterColumn("newString")->Default(KuduValue::CopyString("ocat"));
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  InsertRows(2, 1);
-  ASSERT_OK(tablet_peer_->tablet()->Flush());
-
-  ScanToStrings(&rows);
-  ASSERT_EQ(3, rows.size());
-  EXPECT_EQ("(int32 c0=0, int32 c1=0, int32 newInt32=NULL, string newString=NULL)", rows[0]);
-  EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 newInt32=12345, string newString=\"taco\")",
-            rows[1]);
-  EXPECT_EQ("(int32 c0=33554432, int32 c1=2, int32 newInt32=54321, string newString=\"ocat\")",
-            rows[2]);
-
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("newInt32")->RemoveDefault();
-    table_alterer->AlterColumn("newString")->RemoveDefault();
-    // Add an extra rename step
-    table_alterer->AlterColumn("newString")->RenameTo("newNewString");
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  InsertRows(3, 1);
-  ASSERT_OK(tablet_peer_->tablet()->Flush());
-
-  ScanToStrings(&rows);
-  ASSERT_EQ(4, rows.size());
-  EXPECT_EQ("(int32 c0=0, int32 c1=0, int32 newInt32=NULL, string newNewString=NULL)", rows[0]);
-  EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 newInt32=12345, string newNewString=\"taco\")",
-            rows[1]);
-  EXPECT_EQ("(int32 c0=33554432, int32 c1=2, int32 newInt32=54321, string newNewString=\"ocat\")",
-            rows[2]);
-  EXPECT_EQ("(int32 c0=50331648, int32 c1=3, int32 newInt32=NULL, string newNewString=NULL)",
-            rows[3]);
-
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->DropColumn("newInt32");
-    table_alterer->DropColumn("newNewString");
-    ASSERT_OK(table_alterer->Alter());
-  }
-}
-
-// Test for a bug seen when an alter table creates an empty RLE block and the
-// block is subsequently scanned.
-TEST_F(AlterTableTest, TestAlterEmptyRLEBlock) {
-  // Add an RLE-encoded column
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AddColumn("rle")->Type(KuduColumnSchema::INT32)
-      ->Encoding(client::KuduColumnStorageAttributes::RLE);
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  // Insert some rows
-  InsertRows(0, 3);
-
-  // Now alter the column
-  {
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("rle")->RenameTo("new_rle");
-    ASSERT_OK(table_alterer->Alter());
-  }
-
-  // Now scan the table, which would trigger a CHECK failure
-  // on trying to seek to position 0 in an empty RLE block
-  {
-    vector<string> rows;
-    ScanToStrings(&rows);
-    EXPECT_EQ("(int32 c0=0, int32 c1=0, int32 new_rle=NULL)", rows[0]);
-    EXPECT_EQ("(int32 c0=16777216, int32 c1=1, int32 new_rle=NULL)", rows[1]);
-    EXPECT_EQ("(int32 c0=33554432, int32 c1=2, int32 new_rle=NULL)", rows[2]);
-  }
-}
-
 // Verify that, if a tablet server is down when an alter command is issued,
 // it will eventually receive the command when it restarts.
 TEST_F(AlterTableTest, TestAlterOnTSRestart) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 2c7849d..7923162 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1348,19 +1348,19 @@ Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& current_pb,
         }
 
         if (cur_schema.is_key_column(step.drop_column().name())) {
-          return Status::InvalidArgument("cannot remove a key column",
-                                         step.drop_column().name());
+          return Status::InvalidArgument("cannot remove a key column");
         }
 
         RETURN_NOT_OK(builder.RemoveColumn(step.drop_column().name()));
         break;
       }
-      // Remains for backwards compatibility
+
       case AlterTableRequestPB::RENAME_COLUMN: {
         if (!step.has_rename_column()) {
           return Status::InvalidArgument("RENAME_COLUMN missing column info");
         }
-        // TODO(wdb): In theory we can rename a key (KUDU-1626).
+
+        // TODO: In theory we can rename a key
         if (cur_schema.is_key_column(step.rename_column().old_name())) {
           return Status::InvalidArgument("cannot rename a key column");
         }
@@ -1370,18 +1370,9 @@ Status CatalogManager::ApplyAlterSchemaSteps(const SysTablesEntryPB& current_pb,
                         step.rename_column().new_name()));
         break;
       }
-      case AlterTableRequestPB::ALTER_COLUMN: {
-        if (!step.has_alter_column()) {
-          return Status::InvalidArgument("ALTER_COLUMN missing column info");
-        }
-        const ColumnSchemaDelta col_delta = ColumnSchemaDeltaFromPB(step.alter_column().delta());
-        // TODO(wdb): In theory we can rename a key (KUDU-1626).
-        if (cur_schema.is_key_column(col_delta.name)) {
-          return Status::InvalidArgument("cannot alter a key column", col_delta.name);
-        }
-        RETURN_NOT_OK(builder.ApplyColumnSchemaDelta(col_delta));
-        break;
-      }
+
+      // TODO: EDIT_COLUMN
+
       default: {
         return Status::InvalidArgument("Invalid alter schema step type", step.ShortDebugString());
       }
@@ -1574,8 +1565,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
     switch (step.type()) {
       case AlterTableRequestPB::ADD_COLUMN:
       case AlterTableRequestPB::DROP_COLUMN:
-      case AlterTableRequestPB::RENAME_COLUMN: // Fallthrough intended.
-      case AlterTableRequestPB::ALTER_COLUMN: {
+      case AlterTableRequestPB::RENAME_COLUMN: {
         alter_schema_steps.emplace_back(step);
         break;
       }
@@ -1584,6 +1574,7 @@ Status CatalogManager::AlterTable(const AlterTableRequestPB* req,
         alter_partitioning_steps.emplace_back(step);
         break;
       }
+      case AlterTableRequestPB::ALTER_COLUMN:
       case AlterTableRequestPB::UNKNOWN: {
         return Status::InvalidArgument("Invalid alter step type", step.ShortDebugString());
       }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fe4f157b/src/kudu/master/master.proto
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index 9dfd0a2..fb02dc1 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -425,8 +425,10 @@ message AlterTableRequestPB {
     UNKNOWN = 0;
     ADD_COLUMN = 1;
     DROP_COLUMN = 2;
-    // Deprecated. Replaced by AlterColumn.
     RENAME_COLUMN = 3;
+
+    // TODO(KUDU-861): this will subsume RENAME_COLUMN, but not yet implemented
+    // on the master side.
     ALTER_COLUMN = 4;
     ADD_RANGE_PARTITION = 5;
     DROP_RANGE_PARTITION = 6;
@@ -441,15 +443,11 @@ message AlterTableRequestPB {
     // Name of the column to drop.
     required string name = 1;
   }
-  // Deprecated. Replaced by AlterColumn.
   message RenameColumn {
     // Name of the column to rename;
     required string old_name = 1;
     required string new_name = 2;
   }
-  message AlterColumn {
-    required ColumnSchemaDeltaPB delta = 1;
-  }
   message AddRangePartition {
     // A set of row operations containing the lower and upper range bound for
     // the range partition to add or drop.
@@ -470,7 +468,6 @@ message AlterTableRequestPB {
     optional RenameColumn rename_column = 4;
     optional AddRangePartition add_range_partition = 5;
     optional DropRangePartition drop_range_partition = 6;
-    optional AlterColumn alter_column = 7;
   }
 
   required TableIdentifierPB table = 1;