You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/04/05 01:00:36 UTC

[kudu] 03/03: java/c++: ColumnSchema supports storing column comment

This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit e2870925f9a07da43658447248a4e64acde398b4
Author: helifu <hz...@corp.netease.com>
AuthorDate: Thu Apr 4 15:45:35 2019 +0800

    java/c++: ColumnSchema supports storing column comment
    
    This patch intends to add a new api for storing column
    comment in java client. And according to the comments
    of the reviews, the empty comment is defined to delete
    the comment for the uniform behavior.
    
    In addition, the server-side has already been implemented
    in: https://gerrit.cloudera.org/#/c/12849/
    
    Change-Id: I4cdca4101898062cfe154c15ca40c5943d0e343c
    Reviewed-on: http://gerrit.cloudera.org:8080/12890
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 .../main/java/org/apache/kudu/ColumnSchema.java    | 33 ++++++++++++++--
 .../org/apache/kudu/client/AlterTableOptions.java  | 28 +++++++++++++-
 .../java/org/apache/kudu/client/Operation.java     |  5 ++-
 .../org/apache/kudu/client/ProtobufHelper.java     | 32 ++++++++++++++-
 .../java/org/apache/kudu/TestColumnSchema.java     | 10 +++++
 .../java/org/apache/kudu/client/TestKuduTable.java | 45 ++++++++++++++++++++++
 src/kudu/client/client-test.cc                     | 40 ++++++++++++-------
 src/kudu/client/schema.cc                          | 20 +++++-----
 src/kudu/client/schema.h                           |  8 ++--
 src/kudu/common/schema.cc                          |  2 +-
 src/kudu/common/schema.h                           | 19 +++------
 src/kudu/common/wire_protocol.cc                   | 16 ++++----
 src/kudu/master/catalog_manager.cc                 |  7 ++--
 src/kudu/server/webui_util.cc                      |  4 +-
 14 files changed, 201 insertions(+), 68 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/ColumnSchema.java b/java/kudu-client/src/main/java/org/apache/kudu/ColumnSchema.java
index 881339b..1a69f33 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/ColumnSchema.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/ColumnSchema.java
@@ -44,6 +44,7 @@ public class ColumnSchema {
   private final ColumnTypeAttributes typeAttributes;
   private final int typeSize;
   private final Common.DataType wireType;
+  private final String comment;
 
   /**
    * Specifies the encoding of data for a column on disk.
@@ -102,7 +103,8 @@ public class ColumnSchema {
   private ColumnSchema(String name, Type type, boolean key, boolean nullable,
                        Object defaultValue, int desiredBlockSize, Encoding encoding,
                        CompressionAlgorithm compressionAlgorithm,
-                       ColumnTypeAttributes typeAttributes, Common.DataType wireType) {
+                       ColumnTypeAttributes typeAttributes, Common.DataType wireType,
+                       String comment) {
     this.name = name;
     this.type = type;
     this.key = key;
@@ -114,6 +116,7 @@ public class ColumnSchema {
     this.typeAttributes = typeAttributes;
     this.typeSize = type.getSize(typeAttributes);
     this.wireType = wireType;
+    this.comment = comment;
   }
 
   /**
@@ -204,6 +207,13 @@ public class ColumnSchema {
     return typeSize;
   }
 
+  /**
+   * Return the comment for the column. An empty string means there is no comment.
+   */
+  public String getComment() {
+    return comment;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
@@ -212,12 +222,13 @@ public class ColumnSchema {
     return Objects.equals(name, that.name) &&
         Objects.equals(type, that.type) &&
         Objects.equals(key, that.key) &&
-        Objects.equals(typeAttributes, that.typeAttributes);
+        Objects.equals(typeAttributes, that.typeAttributes) &&
+        Objects.equals(comment, that.comment);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(name, type, key, typeAttributes);
+    return Objects.hash(name, type, key, typeAttributes, comment);
   }
 
   @Override
@@ -230,6 +241,10 @@ public class ColumnSchema {
     if (typeAttributes != null) {
       sb.append(typeAttributes.toStringForType(type));
     }
+    if (!comment.isEmpty()) {
+      sb.append(", comment: ");
+      sb.append(comment);
+    }
     return sb.toString();
   }
 
@@ -249,6 +264,7 @@ public class ColumnSchema {
     private CompressionAlgorithm compressionAlgorithm = null;
     private ColumnTypeAttributes typeAttributes = null;
     private Common.DataType wireType = null;
+    private String comment = "";
 
     /**
      * Constructor for the required parameters.
@@ -275,6 +291,7 @@ public class ColumnSchema {
       this.compressionAlgorithm = that.compressionAlgorithm;
       this.typeAttributes = that.typeAttributes;
       this.wireType = that.wireType;
+      this.comment = that.comment;
     }
 
     /**
@@ -381,6 +398,14 @@ public class ColumnSchema {
     }
 
     /**
+     * Set the comment for this column.
+     */
+    public ColumnSchemaBuilder comment(String comment) {
+      this.comment = comment;
+      return this;
+    }
+
+    /**
      * Builds a {@link ColumnSchema} using the passed parameters.
      * @return a new {@link ColumnSchema}
      */
@@ -392,7 +417,7 @@ public class ColumnSchema {
       return new ColumnSchema(name, type,
                               key, nullable, defaultValue,
                               desiredBlockSize, encoding, compressionAlgorithm,
-                              typeAttributes, wireType);
+                              typeAttributes, wireType, comment);
     }
   }
 }
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 4771c61..677df58 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,6 +17,8 @@
 
 package org.apache.kudu.client;
 
+import java.util.EnumSet;
+
 import static org.apache.kudu.ColumnSchema.CompressionAlgorithm;
 import static org.apache.kudu.ColumnSchema.Encoding;
 import static org.apache.kudu.master.Master.AlterTableRequestPB;
@@ -29,6 +31,7 @@ import org.apache.yetus.audience.InterfaceStability;
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.Common;
 import org.apache.kudu.Type;
+import org.apache.kudu.client.ProtobufHelper.SchemaPBConversionFlags;
 
 /**
  * This builder must be used to alter a table. At least one change must be specified.
@@ -292,7 +295,8 @@ public class AlterTableOptions {
             .encodeLowerAndUpperBounds(lowerBound, upperBound, lowerBoundType, upperBoundType));
     step.setAddRangePartition(builder);
     if (!pb.hasSchema()) {
-      pb.setSchema(ProtobufHelper.schemaToPb(lowerBound.getSchema()));
+      pb.setSchema(ProtobufHelper.schemaToPb(lowerBound.getSchema(),
+          EnumSet.of(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT)));
     }
     return this;
   }
@@ -351,12 +355,32 @@ public class AlterTableOptions {
                                                                     upperBoundType));
     step.setDropRangePartition(builder);
     if (!pb.hasSchema()) {
-      pb.setSchema(ProtobufHelper.schemaToPb(lowerBound.getSchema()));
+      pb.setSchema(ProtobufHelper.schemaToPb(lowerBound.getSchema(),
+          EnumSet.of(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT)));
     }
     return this;
   }
 
   /**
+   * Change the comment for the column.
+   *
+   * @param name name of the column
+   * @param comment the new comment for the column, an empty comment means
+   *        deleting an existing comment.
+   * @return this instance
+   */
+  public AlterTableOptions changeComment(String name, String comment) {
+    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).setNewComment(comment));
+    step.setAlterColumn(alterBuilder);
+    return this;
+  }
+
+  /**
    * Whether to wait for the table to be fully altered before this alter
    * operation is considered to be finished.
    * <p>
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
index 200e8e2..f7b69fb 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Operation.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.ArrayList;
 import java.util.BitSet;
+import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.base.Preconditions;
@@ -38,6 +39,7 @@ import org.apache.kudu.Schema;
 import org.apache.kudu.Type;
 import org.apache.kudu.WireProtocol;
 import org.apache.kudu.WireProtocol.RowOperationsPB;
+import org.apache.kudu.client.ProtobufHelper.SchemaPBConversionFlags;
 import org.apache.kudu.client.Statistics.Statistic;
 import org.apache.kudu.client.Statistics.TabletStatistics;
 import org.apache.kudu.tserver.Tserver;
@@ -279,7 +281,8 @@ public abstract class Operation extends KuduRpc<OperationResponse> {
     }
 
     Tserver.WriteRequestPB.Builder requestBuilder = Tserver.WriteRequestPB.newBuilder();
-    requestBuilder.setSchema(ProtobufHelper.schemaToPb(schema));
+    requestBuilder.setSchema(ProtobufHelper.schemaToPb(schema,
+        EnumSet.of(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT)));
     requestBuilder.setRowOperations(rowOps);
     return requestBuilder;
   }
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 bbb58ce..ed3263d 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
@@ -23,6 +23,7 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 
 import com.google.common.base.Joiner;
@@ -42,24 +43,41 @@ import org.apache.kudu.util.DecimalUtil;
 public class ProtobufHelper {
 
   /**
+   * The flags that are not included while serializing.
+   */
+  public enum SchemaPBConversionFlags {
+    SCHEMA_PB_WITHOUT_COMMENT;
+  }
+
+  /**
    * Utility method to convert a Schema to its wire format.
    * @param schema Schema to convert
    * @return a list of ColumnSchemaPB
    */
   public static List<Common.ColumnSchemaPB> schemaToListPb(Schema schema) {
+    return schemaToListPb(schema, EnumSet.noneOf(SchemaPBConversionFlags.class));
+  }
+
+  public static List<Common.ColumnSchemaPB> schemaToListPb(
+      Schema schema, EnumSet<SchemaPBConversionFlags> flags) {
     ArrayList<Common.ColumnSchemaPB> columns =
         new ArrayList<Common.ColumnSchemaPB>(schema.getColumnCount());
     Common.ColumnSchemaPB.Builder schemaBuilder = Common.ColumnSchemaPB.newBuilder();
     for (ColumnSchema col : schema.getColumns()) {
-      columns.add(columnToPb(schemaBuilder, col));
+      columns.add(columnToPb(schemaBuilder, col, flags));
       schemaBuilder.clear();
     }
     return columns;
   }
 
   public static Common.SchemaPB schemaToPb(Schema schema) {
+    return schemaToPb(schema, EnumSet.noneOf(SchemaPBConversionFlags.class));
+  }
+
+  public static Common.SchemaPB schemaToPb(
+      Schema schema, EnumSet<SchemaPBConversionFlags> flags) {
     Common.SchemaPB.Builder builder = Common.SchemaPB.newBuilder();
-    builder.addAllColumns(schemaToListPb(schema));
+    builder.addAllColumns(schemaToListPb(schema, flags));
     return builder.build();
   }
 
@@ -69,6 +87,11 @@ public class ProtobufHelper {
 
   public static Common.ColumnSchemaPB columnToPb(Common.ColumnSchemaPB.Builder schemaBuilder,
                                                  ColumnSchema column) {
+    return columnToPb(schemaBuilder, column, EnumSet.noneOf(SchemaPBConversionFlags.class));
+  }
+
+  public static Common.ColumnSchemaPB columnToPb(Common.ColumnSchemaPB.Builder schemaBuilder,
+      ColumnSchema column, EnumSet<SchemaPBConversionFlags> flags) {
     schemaBuilder
         .setName(column.getName())
         .setType(column.getWireType())
@@ -90,6 +113,10 @@ public class ProtobufHelper {
       schemaBuilder.setTypeAttributes(
           columnTypeAttributesToPb(Common.ColumnTypeAttributesPB.newBuilder(), column));
     }
+    if (!flags.contains(SchemaPBConversionFlags.SCHEMA_PB_WITHOUT_COMMENT) &&
+        !column.getComment().isEmpty()) {
+      schemaBuilder.setComment(column.getComment());
+    }
     return schemaBuilder.build();
   }
 
@@ -123,6 +150,7 @@ public class ProtobufHelper {
                            .compressionAlgorithm(compressionAlgorithm)
                            .desiredBlockSize(desiredBlockSize)
                            .typeAttributes(typeAttributes)
+                           .comment(pb.getComment())
                            .build();
   }
 
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java b/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
index d227f45..3f85898 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/TestColumnSchema.java
@@ -18,6 +18,7 @@ package org.apache.kudu;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.fail;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -38,10 +39,12 @@ public class TestColumnSchema {
     ColumnSchema col3 = new ColumnSchemaBuilder("col3", Type.DECIMAL)
         .typeAttributes(DecimalUtil.typeAttributes(5, 2))
         .build();
+    ColumnSchema col4 = new ColumnSchemaBuilder("col4", Type.INT16).comment("test comment").build();
 
     assertEquals("Column name: col1, type: string", col1.toString());
     assertEquals("Column name: col2, type: int64", col2.toString());
     assertEquals("Column name: col3, type: decimal(5, 2)", col3.toString());
+    assertEquals("Column name: col4, type: int16, comment: test comment", col4.toString());
   }
 
   @Test
@@ -84,6 +87,13 @@ public class TestColumnSchema {
         .build();
     assertNotEquals(decCol1, decCol3);
 
+    // Same with comment
+    ColumnSchema commentInt1 = new ColumnSchemaBuilder("col1", Type.INT32).comment("test").build();
+    ColumnSchema commentInt2 = new ColumnSchemaBuilder("col1", Type.INT32).comment("test").build();
+    assertEquals(commentInt1, commentInt2);
 
+    // Different by comment
+    ColumnSchema commentInt3 = new ColumnSchemaBuilder("col1", Type.INT32).comment("Test").build();
+    assertNotEquals(commentInt1, commentInt3);
   }
 }
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 d49fdda..a3d3d2c 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
@@ -701,4 +701,49 @@ public class TestKuduTable {
       }
     }
   }
+
+  @Test(timeout = 100000)
+  public void testAlterColumnComment() throws Exception {
+    // Schema with comments.
+    List<ColumnSchema> columns = ImmutableList.of(
+        new ColumnSchema.ColumnSchemaBuilder("key", Type.INT32)
+            .key(true).comment("keytest").build(),
+        new ColumnSchema.ColumnSchemaBuilder("value", Type.STRING)
+            .comment("valuetest").build());
+    // Create a table.
+    KuduTable table = client.createTable(tableName,
+        new Schema(columns), getBasicCreateTableOptions());
+
+    // Verify the comments after creating.
+    assertEquals("wrong key comment", "keytest",
+        table.getSchema().getColumn("key").getComment());
+    assertEquals("wrong value comment", "valuetest",
+        table.getSchema().getColumn("value").getComment());
+
+    // Change the comments.
+    client.alterTable(tableName,
+        new AlterTableOptions().changeComment("key", "keycomment"));
+    client.alterTable(tableName,
+        new AlterTableOptions().changeComment("value", "valuecomment"));
+
+    // Verify the comments after the first change.
+    KuduTable table1 = client.openTable(tableName);
+    assertEquals("wrong key comment post alter",
+        "keycomment", table1.getSchema().getColumn("key").getComment());
+    assertEquals("wrong value comment post alter",
+        "valuecomment", table1.getSchema().getColumn("value").getComment());
+
+    // Delete the comments.
+    client.alterTable(tableName,
+        new AlterTableOptions().changeComment("key", ""));
+    client.alterTable(tableName,
+        new AlterTableOptions().changeComment("value", ""));
+
+    // Verify the comments after the second change.
+    KuduTable table2 = client.openTable(tableName);
+    assertEquals("wrong key comment post alter", "",
+        table2.getSchema().getColumn("key").getComment());
+    assertEquals("wrong value comment post alter", "",
+        table2.getSchema().getColumn("value").getComment());
+  }
 }
diff --git a/src/kudu/client/client-test.cc b/src/kudu/client/client-test.cc
index a75fd6d..117a8c3 100644
--- a/src/kudu/client/client-test.cc
+++ b/src/kudu/client/client-test.cc
@@ -4915,19 +4915,33 @@ TEST_F(ClientTest, TestCreateTableWithValidComment) {
 TEST_F(ClientTest, TestAlterTableWithValidComment) {
   const string kTableName = "table_comment";
   const auto AlterAndVerify = [&] (int i) {
-    // The comment length should be less and equal than FLAGS_max_column_comment_length.
-    string kLongComment(i * 16, 'x');
-
-    // Alter the table with comment.
-    unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
-    table_alterer->AlterColumn("val")->Comment(kLongComment);
-    ASSERT_OK(table_alterer->Alter());
-
-    // Open the table and verify the comment.
-    KuduSchema schema;
-    ASSERT_OK(client_->GetTableSchema(kTableName, &schema));
-    ASSERT_EQ(schema.Column(1).name(), "val");
-    ASSERT_EQ(schema.Column(1).comment(), kLongComment);
+    {
+      // The comment length should be less and equal than FLAGS_max_column_comment_length.
+      string kLongComment(i * 16, 'x');
+
+      // Alter the table with comment.
+      unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+      table_alterer->AlterColumn("val")->Comment(kLongComment);
+      ASSERT_OK(table_alterer->Alter());
+
+      // Open the table and verify the comment.
+      KuduSchema schema;
+      ASSERT_OK(client_->GetTableSchema(kTableName, &schema));
+      ASSERT_EQ(schema.Column(1).name(), "val");
+      ASSERT_EQ(schema.Column(1).comment(), kLongComment);
+    }
+    {
+      // Delete the comment.
+      unique_ptr<KuduTableAlterer> table_alterer(client_->NewTableAlterer(kTableName));
+      table_alterer->AlterColumn("val")->Comment("");
+      ASSERT_OK(table_alterer->Alter());
+
+      // Open the table and verify the comment.
+      KuduSchema schema;
+      ASSERT_OK(client_->GetTableSchema(kTableName, &schema));
+      ASSERT_EQ(schema.Column(1).name(), "val");
+      ASSERT_EQ(schema.Column(1).comment(), "");
+    }
   };
 
   // Create a table.
diff --git a/src/kudu/client/schema.cc b/src/kudu/client/schema.cc
index d6b90d2..78113fb 100644
--- a/src/kudu/client/schema.cc
+++ b/src/kudu/client/schema.cc
@@ -368,7 +368,8 @@ Status KuduColumnSpec::ToColumnSchema(KuduColumnSchema* col) const {
   *col = KuduColumnSchema(data_->name, data_->type.value(), nullable,
                           default_val,
                           KuduColumnStorageAttributes(encoding, compression, block_size),
-                          type_attrs, data_->comment.get_ptr());
+                          type_attrs,
+                          data_->comment ? data_->comment.value() : "");
 #pragma GCC diagnostic pop
 
   return Status::OK();
@@ -388,9 +389,9 @@ Status KuduColumnSpec::ToColumnSchemaDelta(ColumnSchemaDelta* col_delta) const {
   if (data_->remove_default && data_->default_val) {
     return Status::InvalidArgument("new default set but default also removed", data_->name);
   }
-
-  if (data_->default_val) {
-    col_delta->default_value = DefaultValueAsSlice();
+
+  if (data_->default_val) {
+    col_delta->default_value = DefaultValueAsSlice();
   }
 
   if (data_->remove_default) {
@@ -584,7 +585,7 @@ KuduColumnSchema::KuduColumnSchema(const string &name,
                                    const void* default_value,
                                    const KuduColumnStorageAttributes& storage_attributes,
                                    const KuduColumnTypeAttributes& type_attributes,
-                                   const string* comment) {
+                                   const string& comment) {
   ColumnStorageAttributes attr_private;
   attr_private.encoding = ToInternalEncodingType(storage_attributes.encoding());
   attr_private.compression = ToInternalCompressionType(
@@ -595,8 +596,7 @@ KuduColumnSchema::KuduColumnSchema(const string &name,
   col_ = new ColumnSchema(name, ToInternalDataType(type, type_attributes),
                           is_nullable,
                           default_value, default_value, attr_private,
-                          type_attr_private,
-                          comment ? boost::optional<string>(*comment) : boost::none);
+                          type_attr_private, comment);
 }
 
 KuduColumnSchema::KuduColumnSchema(const KuduColumnSchema& other)
@@ -650,8 +650,8 @@ KuduColumnTypeAttributes KuduColumnSchema::type_attributes() const {
   return KuduColumnTypeAttributes(type_attributes.precision, type_attributes.scale);
 }
 
-string KuduColumnSchema::comment() const {
-  return DCHECK_NOTNULL(col_)->comment() ? *col_->comment() : "";
+const string& KuduColumnSchema::comment() const {
+  return DCHECK_NOTNULL(col_)->comment();
 }
 
 ////////////////////////////////////////////////////////////
@@ -715,7 +715,7 @@ KuduColumnSchema KuduSchema::Column(size_t idx) const {
   KuduColumnTypeAttributes type_attrs(col.type_attributes().precision, col.type_attributes().scale);
   return KuduColumnSchema(col.name(), FromInternalDataType(col.type_info()->type()),
                           col.is_nullable(), col.read_default_value(),
-                          attrs, type_attrs, col.comment().get_ptr());
+                          attrs, type_attrs, col.comment());
 }
 
 KuduPartialRow* KuduSchema::NewRow() const {
diff --git a/src/kudu/client/schema.h b/src/kudu/client/schema.h
index b59fa39..10c7578 100644
--- a/src/kudu/client/schema.h
+++ b/src/kudu/client/schema.h
@@ -244,9 +244,8 @@ class KUDU_EXPORT KuduColumnSchema {
 
   /// @return comment of the column schema.
   ///
-  /// @note Both columns with no comments and empty comments will return
-  ///   empty strings here.
-  std::string comment() const;
+  /// @note An empty string will be returned if there is no comment.
+  const std::string& comment() const;
 
  private:
   friend class KuduColumnSpec;
@@ -275,8 +274,7 @@ class KUDU_EXPORT KuduColumnSchema {
       const void* default_value = NULL, //NOLINT(modernize-use-nullptr)
       const KuduColumnStorageAttributes& storage_attributes = KuduColumnStorageAttributes(),
       const KuduColumnTypeAttributes& type_attributes = KuduColumnTypeAttributes(),
-      const std::string* comment = NULL //NOLINT(modernize-use-nullptr)
-      );
+      const std::string& comment = "");
 #if defined(__clang__) || \
   (defined(__GNUC__) && (__GNUC__ * 10000 + __GNUC_MINOR__ * 100) >= 40600)
 #pragma GCC diagnostic pop
diff --git a/src/kudu/common/schema.cc b/src/kudu/common/schema.cc
index 81ba0a2..074689a 100644
--- a/src/kudu/common/schema.cc
+++ b/src/kudu/common/schema.cc
@@ -116,7 +116,7 @@ Status ColumnSchema::ApplyDelta(const ColumnSchemaDelta& col_delta) {
     attributes_.cfile_block_size = *col_delta.cfile_block_size;
   }
   if (col_delta.new_comment) {
-    comment_ = col_delta.new_comment;
+    comment_ = col_delta.new_comment.value();
   }
   return Status::OK();
 }
diff --git a/src/kudu/common/schema.h b/src/kudu/common/schema.h
index 89dbdd9..610897d 100644
--- a/src/kudu/common/schema.h
+++ b/src/kudu/common/schema.h
@@ -206,7 +206,7 @@ class ColumnSchema {
                const void* write_default = nullptr,
                ColumnStorageAttributes attributes = ColumnStorageAttributes(),
                ColumnTypeAttributes type_attributes = ColumnTypeAttributes(),
-               boost::optional<std::string> comment = boost::none)
+               std::string comment = "")
       : name_(std::move(name)),
         type_info_(GetTypeInfo(type)),
         is_nullable_(is_nullable),
@@ -253,7 +253,7 @@ class ColumnSchema {
   // For example, "AUTO_ENCODING ZLIB 123 123".
   std::string AttrToString() const;
 
-  const boost::optional<std::string>& comment() const {
+  const std::string& comment() const {
     return comment_;
   }
 
@@ -344,17 +344,8 @@ class ColumnSchema {
       if (write_default_ != nullptr && !write_default_->Equals(other.write_default_.get()))
         return false;
 
-      // "no comment" and "empty comment" are the same.
-      if (comment_) {
-        if (other.comment_) {
-          if (*comment_ != *other.comment_) return false;
-        } else {
-          if (!comment_->empty()) return false;
-        }
-      } else {
-        if (other.comment_) {
-          if (!other.comment_->empty()) return false;
-        }
+      if (comment_ != other.comment_) {
+        return false;
       }
     }
     return true;
@@ -427,7 +418,7 @@ class ColumnSchema {
   std::shared_ptr<Variant> write_default_;
   ColumnStorageAttributes attributes_;
   ColumnTypeAttributes type_attributes_;
-  boost::optional<std::string> comment_;
+  std::string comment_;
 };
 
 // The schema for a set of rows.
diff --git a/src/kudu/common/wire_protocol.cc b/src/kudu/common/wire_protocol.cc
index fa30e15..ce2cf0b 100644
--- a/src/kudu/common/wire_protocol.cc
+++ b/src/kudu/common/wire_protocol.cc
@@ -23,7 +23,6 @@
 #include <cstring>
 #include <ostream>
 #include <string>
-#include <utility>
 #include <vector>
 
 #include <boost/optional/optional.hpp>
@@ -247,8 +246,8 @@ void ColumnSchemaToPB(const ColumnSchema& col_schema, ColumnSchemaPB *pb, int fl
       pb->set_write_default_value(write_value, col_schema.type_info()->size());
     }
   }
-  if (col_schema.comment() && !(flags & SCHEMA_PB_WITHOUT_COMMENT)) {
-    pb->set_comment(*col_schema.comment());
+  if (!col_schema.comment().empty() && !(flags & SCHEMA_PB_WITHOUT_COMMENT)) {
+    pb->set_comment(col_schema.comment());
   }
 }
 
@@ -297,14 +296,13 @@ ColumnSchema ColumnSchemaFromPB(const ColumnSchemaPB& pb) {
     attributes.cfile_block_size = pb.cfile_block_size();
   }
 
-  boost::optional<string> comment;
-  if (pb.has_comment()) {
-    comment = boost::optional<string>(pb.comment());
-  }
-
+  // According to the URL below, the default value for strings that are optional
+  // in protobuf is the empty string. So, it's safe to use pb.comment() directly
+  // regardless of whether has_comment() is true or false.
+  // https://developers.google.com/protocol-buffers/docs/proto#optional
   return ColumnSchema(pb.name(), pb.type(), pb.is_nullable(),
                       read_default_ptr, write_default_ptr,
-                      attributes, type_attributes, std::move(comment));
+                      attributes, type_attributes, pb.comment());
 }
 
 void ColumnSchemaDeltaToPB(const ColumnSchemaDelta& col_delta, ColumnSchemaDeltaPB *pb) {
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 3a2f9b2..b3480ea 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -1338,13 +1338,12 @@ Status ValidateIdentifier(const string& id) {
 }
 
 // Validate a column comment to ensure that it is a valid identifier.
-Status ValidateCommentIdentifier(const boost::optional<string>& id) {
-  // It's safe here because logical OR has left-to-right associativity.
-  if (!id || id->empty()) {
+Status ValidateCommentIdentifier(const string& id) {
+  if (id.empty()) {
     return Status::OK();
   }
 
-  return ValidateLengthAndUTF8(*id, FLAGS_max_column_comment_length);
+  return ValidateLengthAndUTF8(id, FLAGS_max_column_comment_length);
 }
 
 // Validate the client-provided schema and name.
diff --git a/src/kudu/server/webui_util.cc b/src/kudu/server/webui_util.cc
index 44a45be..376ec72 100644
--- a/src/kudu/server/webui_util.cc
+++ b/src/kudu/server/webui_util.cc
@@ -19,8 +19,6 @@
 
 #include <string>
 
-#include <boost/optional/optional.hpp>
-
 #include "kudu/common/common.pb.h"
 #include "kudu/common/schema.h"
 #include "kudu/gutil/ref_counted.h"
@@ -52,7 +50,7 @@ void SchemaToJson(const Schema& schema, EasyJson* output) {
                                    col.Stringify(col.read_default_value()) : "-";
     col_json["write_default"] = col.has_write_default() ?
                                     col.Stringify(col.write_default_value()) : "-";
-    col_json["comment"] = col.comment() ? *col.comment() : "-";
+    col_json["comment"] = col.comment();
   }
 }