You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/06 18:01:33 UTC

[kudu] branch master updated (7c8797f -> ef1e772)

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

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


    from 7c8797f  [test] Fix bug of TestSplitKeyRange
     new 3f48094  [hms] disallow table type altering via table property
     new 6a14790  [java] Add newComparisonPredicate Object API
     new ef1e772  [java] Add PartialRow addObject API

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/kudu/client/KuduPredicate.java | 55 +++++++++++++
 .../java/org/apache/kudu/client/PartialRow.java    | 95 ++++++++++++++++++++++
 .../org/apache/kudu/client/TestKuduPredicate.java  | 33 ++++++++
 .../org/apache/kudu/client/TestPartialRow.java     | 63 ++++++++------
 .../kudu/hive/metastore/KuduMetastorePlugin.java   | 31 +++++--
 .../hive/metastore/TestKuduMetastorePlugin.java    | 18 ++++
 .../org/apache/kudu/spark/kudu/DefaultSource.scala | 28 +------
 .../java/org/apache/kudu/test/ClientTestUtil.java  | 28 +++++++
 8 files changed, 296 insertions(+), 55 deletions(-)


[kudu] 01/03: [hms] disallow table type altering via table property

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3f480941156068342dc2c66e38553712623d959e
Author: Hao Hao <ha...@cloudera.com>
AuthorDate: Wed Jun 5 15:20:00 2019 -0700

    [hms] disallow table type altering via table property
    
    This blocks table type altering for Kudu table even via altering table
    property 'EXTERNAL', to avoid orphaned tables.
    
    Change-Id: I7c6d490f7714770c6544859ca8477e6e765a06b3
    Reviewed-on: http://gerrit.cloudera.org:8080/13524
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
    Tested-by: Grant Henke <gr...@apache.org>
---
 .../kudu/hive/metastore/KuduMetastorePlugin.java   | 31 ++++++++++++++++++----
 .../hive/metastore/TestKuduMetastorePlugin.java    | 18 +++++++++++++
 2 files changed, 44 insertions(+), 5 deletions(-)

diff --git a/java/kudu-hive/src/main/java/org/apache/kudu/hive/metastore/KuduMetastorePlugin.java b/java/kudu-hive/src/main/java/org/apache/kudu/hive/metastore/KuduMetastorePlugin.java
index cb6e5b4..f1a1f19 100644
--- a/java/kudu-hive/src/main/java/org/apache/kudu/hive/metastore/KuduMetastorePlugin.java
+++ b/java/kudu-hive/src/main/java/org/apache/kudu/hive/metastore/KuduMetastorePlugin.java
@@ -80,6 +80,10 @@ public class KuduMetastorePlugin extends MetaStoreEventListener {
   static final String KUDU_MASTER_EVENT = "kudu.master_event";
   @VisibleForTesting
   static final String KUDU_CHECK_ID_KEY = "kudu.check_id";
+  // The key should keep in sync with the one used in
+  // org.apache.hadoop.hive.metastore.MetaStoreUtils.isExternalTable().
+  @VisibleForTesting
+  static final String EXTERNAL_TABLE_KEY= "EXTERNAL";
 
   // System env to track if the HMS plugin validation should be skipped.
   static final String SKIP_VALIDATION_ENV = "KUDU_SKIP_HMS_PLUGIN_VALIDATION";
@@ -165,16 +169,18 @@ public class KuduMetastorePlugin extends MetaStoreEventListener {
     Table oldTable = tableEvent.getOldTable();
     Table newTable = tableEvent.getNewTable();
 
-    // Prevent altering the table type (managed/external) of Kudu tables.
-    // This can cause orphaned tables and the Sentry integration depends on
-    // having a managed table for each Kudu table to prevent security issues
-    // due to overlapping names with Kudu tables and tables in the HMS.
+    // Prevent altering the table type (managed/external) of Kudu tables (or via
+    // altering table property 'EXTERNAL'). This can cause orphaned tables and
+    // the Sentry integration depends on having a managed table for each Kudu
+    // table to prevent security issues due to overlapping names with Kudu tables
+    // and tables in the HMS.
     // Note: This doesn't prevent altering the table type for legacy tables
     // because they should continue to work as they always have primarily for
     // migration purposes.
     String oldTableType = oldTable.getTableType();
     if (isKuduTable(oldTable) &&
-        oldTableType != null && !oldTableType.equals(newTable.getTableType())) {
+        ((oldTableType != null && !oldTableType.equals(newTable.getTableType())) ||
+         (isExternalTable(oldTable) != isExternalTable(newTable)))) {
       throw new MetaException("Kudu table type may not be altered");
     }
 
@@ -249,6 +255,21 @@ public class KuduMetastorePlugin extends MetaStoreEventListener {
   }
 
   /**
+   * Checks whether the table is an external table.
+   *
+   * @param table the table to check
+   * @return {@code true} if the table is an external table,
+   *         otherwise {@code false}
+   */
+  private boolean isExternalTable(Table table) {
+    String isExternal = table.getParameters().get(EXTERNAL_TABLE_KEY);
+    if (isExternal == null) {
+      return false;
+    }
+    return Boolean.parseBoolean(isExternal);
+  }
+
+  /**
    * Checks that the Kudu table entry contains the required Kudu table properties.
    * @param table the table to check
    */
diff --git a/java/kudu-hive/src/test/java/org/apache/kudu/hive/metastore/TestKuduMetastorePlugin.java b/java/kudu-hive/src/test/java/org/apache/kudu/hive/metastore/TestKuduMetastorePlugin.java
index 762d411..7148c1b 100644
--- a/java/kudu-hive/src/test/java/org/apache/kudu/hive/metastore/TestKuduMetastorePlugin.java
+++ b/java/kudu-hive/src/test/java/org/apache/kudu/hive/metastore/TestKuduMetastorePlugin.java
@@ -268,6 +268,24 @@ public class TestKuduMetastorePlugin {
             "Kudu table type may not be altered"));
       }
 
+      // Alter the Kudu table to a different type by setting the table property fails.
+      try {
+        Table alteredTable = table.deepCopy();
+        alteredTable.putToParameters(KuduMetastorePlugin.EXTERNAL_TABLE_KEY, "true");
+        client.alter_table(table.getDbName(), table.getTableName(), alteredTable);
+        fail();
+      } catch (TException e) {
+        assertTrue(e.getMessage().contains(
+           "Kudu table type may not be altered"));
+      }
+
+      // Alter the Kudu table to the same type by setting the table property works.
+      {
+        Table alteredTable = table.deepCopy();
+        alteredTable.putToParameters(KuduMetastorePlugin.EXTERNAL_TABLE_KEY, "false");
+        client.alter_table(table.getDbName(), table.getTableName(), alteredTable);
+      }
+
       // Check that altering the table succeeds.
       client.alter_table(table.getDbName(), table.getTableName(), table);
 


[kudu] 02/03: [java] Add newComparisonPredicate Object API

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6a14790405e8f550fbbd050ad2b61d7e11ae5024
Author: Grant Henke <gr...@apache.org>
AuthorDate: Wed Jun 5 21:38:23 2019 -0500

    [java] Add newComparisonPredicate Object API
    
    Adds a `newComparisonPredicate` API that takes an
    Object in order to support creating comparison
    predicates for Java objects generically.
    
    Change-Id: If05b2d9f61990b86d92af4b89e21704d5188192e
    Reviewed-on: http://gerrit.cloudera.org:8080/13527
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 .../java/org/apache/kudu/client/KuduPredicate.java | 55 ++++++++++++++++++++++
 .../org/apache/kudu/client/TestKuduPredicate.java  | 33 +++++++++++++
 .../org/apache/kudu/spark/kudu/DefaultSource.scala | 28 +----------
 3 files changed, 89 insertions(+), 27 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduPredicate.java b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduPredicate.java
index 762ad08..dc37706 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/KuduPredicate.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/KuduPredicate.java
@@ -22,6 +22,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.math.BigInteger;
 import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -480,6 +481,60 @@ public class KuduPredicate {
   }
 
   /**
+   * Creates a new comparison predicate on a column.
+   *
+   * This method is useful when you don't care about autoboxing
+   * and your existing type handling logic is based on Java types.
+   *
+   * The accepted Object type is based on the column's {@link Type}:
+   *  Type.BOOL -> java.lang.Boolean
+   *  Type.INT8 -> java.lang.Byte
+   *  Type.INT16 -> java.lang.Short
+   *  Type.INT32 -> java.lang.Integer
+   *  Type.INT64 -> java.lang.Long
+   *  Type.UNIXTIME_MICROS -> java.sql.Timestamp or java.lang.Long
+   *  Type.FLOAT -> java.lang.Float
+   *  Type.DOUBLE -> java.lang.Double
+   *  Type.STRING -> java.lang.String
+   *  Type.BINARY -> byte[]
+   *  Type.DECIMAL -> java.math.BigDecimal
+   *
+   * @param column column the column schema
+   * @param op the comparison operation
+   * @param value the value to compare against
+   */
+  public static KuduPredicate newComparisonPredicate(ColumnSchema column,
+                                                     ComparisonOp op,
+                                                     Object value) {
+    if (value instanceof Boolean) {
+      return newComparisonPredicate(column, op, (boolean) value);
+    } else if (value instanceof Byte) {
+      return newComparisonPredicate(column, op, (byte) value);
+    } else if (value instanceof Short) {
+      return newComparisonPredicate(column, op, (short) value);
+    } else if (value instanceof Integer) {
+      return newComparisonPredicate(column, op, (int) value);
+    } else if (value instanceof Long) {
+      return newComparisonPredicate(column, op, (long) value);
+    } else if (value instanceof Timestamp) {
+      return newComparisonPredicate(column, op, (Timestamp) value);
+    } else if (value instanceof Float) {
+      return newComparisonPredicate(column, op, (float) value);
+    } else if (value instanceof Double) {
+      return newComparisonPredicate(column, op, (double) value);
+    } else if (value instanceof BigDecimal) {
+      return newComparisonPredicate(column, op, (BigDecimal) value);
+    } else if (value instanceof String) {
+      return newComparisonPredicate(column, op, (String) value);
+    } else if (value instanceof byte[]) {
+      return newComparisonPredicate(column, op, (byte[]) value);
+    } else {
+      throw new IllegalArgumentException(String.format("illegal type for %s predicate: %s",
+              op, value.getClass().getName()));
+    }
+  }
+
+  /**
    * Creates a new IN list predicate.
    *
    * The list must contain values of the correct type for the column.
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
index 9dcb7bc..7aec32e 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestKuduPredicate.java
@@ -1096,6 +1096,39 @@ public class TestKuduPredicate {
   }
 
   @Test
+  public void testCreateWithObject() {
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(byteCol, EQUAL, (Object) (byte) 10),
+        KuduPredicate.newComparisonPredicate(byteCol, EQUAL, (byte) 10));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(shortCol, EQUAL, (Object) (short) 10),
+        KuduPredicate.newComparisonPredicate(shortCol, EQUAL, 10));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(intCol, EQUAL, (Object) 10),
+        KuduPredicate.newComparisonPredicate(intCol, EQUAL, 10));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(longCol, EQUAL, (Object) 10L),
+        KuduPredicate.newComparisonPredicate(longCol, EQUAL, 10L));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(floatCol, EQUAL, (Object) 12.345f),
+        KuduPredicate.newComparisonPredicate(floatCol, EQUAL, 12.345f));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(doubleCol, EQUAL, (Object) 12.345),
+        KuduPredicate.newComparisonPredicate(doubleCol, EQUAL, 12.345));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(decimal32Col, EQUAL,
+            (Object) BigDecimal.valueOf(12345,2)),
+        KuduPredicate.newComparisonPredicate(decimal32Col, EQUAL,
+            BigDecimal.valueOf(12345,2)));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(stringCol, EQUAL, (Object) "a"),
+        KuduPredicate.newComparisonPredicate(stringCol, EQUAL, "a"));
+    Assert.assertEquals(
+        KuduPredicate.newComparisonPredicate(binaryCol, EQUAL, (Object) new byte[] { (byte) 10 }),
+        KuduPredicate.newComparisonPredicate(binaryCol, EQUAL, new byte[] { (byte) 10 }));
+  }
+
+  @Test
   public void testToString() {
     Assert.assertEquals("`bool` = true",
                         KuduPredicate.newComparisonPredicate(boolCol, EQUAL, true).toString());
diff --git a/java/kudu-spark/src/main/scala/org/apache/kudu/spark/kudu/DefaultSource.scala b/java/kudu-spark/src/main/scala/org/apache/kudu/spark/kudu/DefaultSource.scala
index 4a7470f..fcbafc2 100644
--- a/java/kudu-spark/src/main/scala/org/apache/kudu/spark/kudu/DefaultSource.scala
+++ b/java/kudu-spark/src/main/scala/org/apache/kudu/spark/kudu/DefaultSource.scala
@@ -17,9 +17,7 @@
 
 package org.apache.kudu.spark.kudu
 
-import java.math.BigDecimal
 import java.net.InetAddress
-import java.sql.Timestamp
 
 import scala.collection.JavaConverters._
 import scala.util.Try
@@ -357,31 +355,7 @@ class KuduRelation(
       column: String,
       operator: ComparisonOp,
       value: Any): KuduPredicate = {
-    val columnSchema = table.getSchema.getColumn(column)
-    value match {
-      case value: Boolean =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Byte =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Short =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Int =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Long =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Timestamp =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Float =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Double =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: String =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: Array[Byte] =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-      case value: BigDecimal =>
-        KuduPredicate.newComparisonPredicate(columnSchema, operator, value)
-    }
+    KuduPredicate.newComparisonPredicate(table.getSchema.getColumn(column), operator, value)
   }
 
   /**


[kudu] 03/03: [java] Add PartialRow addObject API

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ef1e772d8d3263fef99fb579a98f491ba6ec02fc
Author: Grant Henke <gr...@apache.org>
AuthorDate: Wed Jun 5 21:57:34 2019 -0500

    [java] Add PartialRow addObject API
    
    Adds a new API to add an Object to a PartialRow. This method is useful when you don't care about autoboxing
    and your existing type handling logic is based on Java types.
    
    Change-Id: I046de9dba8f4bedae02b82632e26c10c313c775c
    Reviewed-on: http://gerrit.cloudera.org:8080/13528
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
---
 .../java/org/apache/kudu/client/PartialRow.java    | 95 ++++++++++++++++++++++
 .../org/apache/kudu/client/TestPartialRow.java     | 63 ++++++++------
 .../java/org/apache/kudu/test/ClientTestUtil.java  | 28 +++++++
 3 files changed, 163 insertions(+), 23 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/PartialRow.java b/java/kudu-client/src/main/java/org/apache/kudu/client/PartialRow.java
index 16055df..d15e0e7 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/PartialRow.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/PartialRow.java
@@ -917,6 +917,101 @@ public class PartialRow {
   }
 
   /**
+   * Add the specified column's value as an Object.
+   *
+   * This method is useful when you don't care about autoboxing
+   * and your existing type handling logic is based on Java types.
+   *
+   * The accepted Object type is based on the column's {@link Type}:
+   *  Type.BOOL -> java.lang.Boolean
+   *  Type.INT8 -> java.lang.Byte
+   *  Type.INT16 -> java.lang.Short
+   *  Type.INT32 -> java.lang.Integer
+   *  Type.INT64 -> java.lang.Long
+   *  Type.UNIXTIME_MICROS -> java.sql.Timestamp or java.lang.Long
+   *  Type.FLOAT -> java.lang.Float
+   *  Type.DOUBLE -> java.lang.Double
+   *  Type.STRING -> java.lang.String
+   *  Type.BINARY -> byte[] or java.lang.ByteBuffer
+   *  Type.DECIMAL -> java.math.BigDecimal
+   *
+   * @param columnName name of the column in the schema
+   * @param val the value to add as an Object
+   * @throws IllegalStateException if the row was already applied
+   * @throws IndexOutOfBoundsException if the column doesn't exist
+   */
+  public void addObject(String columnName, Object val) {
+    addObject(this.schema.getColumnIndex(columnName), val);
+  }
+
+  /**
+   * Add the specified column's value as an Object.
+   *
+   * This method is useful when you don't care about autoboxing
+   * and your existing type handling logic is based on Java types.
+   *
+   * The accepted Object type is based on the column's {@link Type}:
+   *  Type.BOOL -> java.lang.Boolean
+   *  Type.INT8 -> java.lang.Byte
+   *  Type.INT16 -> java.lang.Short
+   *  Type.INT32 -> java.lang.Integer
+   *  Type.INT64 -> java.lang.Long
+   *  Type.UNIXTIME_MICROS -> java.sql.Timestamp or java.lang.Long
+   *  Type.FLOAT -> java.lang.Float
+   *  Type.DOUBLE -> java.lang.Double
+   *  Type.STRING -> java.lang.String
+   *  Type.BINARY -> byte[] or java.lang.ByteBuffer
+   *  Type.DECIMAL -> java.math.BigDecimal
+   *
+   * @param columnIndex column index in the schema
+   * @param val the value to add as an Object
+   * @throws IllegalStateException if the row was already applied
+   * @throws IndexOutOfBoundsException if the column doesn't exist
+   */
+  public void addObject(int columnIndex, Object val) {
+    checkNotFrozen();
+    ColumnSchema col = schema.getColumnByIndex(columnIndex);
+    checkColumnExists(col);
+    try {
+      if (val == null) {
+        setNull(columnIndex);
+        return;
+      }
+      switch (col.getType()) {
+        case BOOL: addBoolean(columnIndex, (Boolean) val); break;
+        case INT8: addByte(columnIndex, (Byte) val); break;
+        case INT16: addShort(columnIndex, (Short) val); break;
+        case INT32: addInt(columnIndex, (Integer) val); break;
+        case INT64: addLong(columnIndex, (Long) val); break;
+        case UNIXTIME_MICROS:
+          if (val instanceof Timestamp) {
+            addTimestamp(columnIndex, (Timestamp) val);
+          } else {
+            addLong(columnIndex, (Long) val);
+          }
+          break;
+        case FLOAT: addFloat(columnIndex, (Float) val); break;
+        case DOUBLE: addDouble(columnIndex, (Double) val); break;
+        case STRING: addString(columnIndex, (String) val); break;
+        case BINARY:
+          if (val instanceof byte[]) {
+            addBinary(columnIndex, (byte[]) val);
+          } else {
+            addBinary(columnIndex, (ByteBuffer) val);
+          }
+          break;
+        case DECIMAL: addDecimal(columnIndex, (BigDecimal) val); break;
+        default:
+          throw new IllegalArgumentException("Unsupported column type: " + col.getType());
+      }
+    } catch (ClassCastException e) {
+      throw new IllegalArgumentException(
+          "Value type does not match column type " + col.getType() +
+              " for column " + col.getName());
+    }
+  }
+
+  /**
    * Get the specified column's value as an Object.
    *
    * This method is useful when you don't care about autoboxing
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
index 3c93c03..f4353dd 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestPartialRow.java
@@ -17,6 +17,7 @@
 
 package org.apache.kudu.client;
 
+import static org.apache.kudu.test.ClientTestUtil.getPartialRowWithAllTypes;
 import static org.apache.kudu.test.ClientTestUtil.getSchemaWithAllTypes;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -93,6 +94,35 @@ public class TestPartialRow {
     assertEquals(BigDecimal.valueOf(12345, 3), partialRow.getObject("decimal"));
   }
 
+  @Test
+  public void testAddObject() {
+    Schema schema = getSchemaWithAllTypes();
+    // Ensure we aren't missing any types
+    assertEquals(13, schema.getColumnCount());
+
+    PartialRow row = schema.newPartialRow();
+    row.addObject("int8", (byte) 42);
+    row.addObject("int16", (short) 43);
+    row.addObject("int32", 44);
+    row.addObject("int64", 45L);
+    row.addObject("timestamp", new Timestamp(1234567890));
+    row.addObject("bool", true);
+    row.addObject("float", 52.35F);
+    row.addObject("double", 53.35);
+    row.addObject("string", "fun with ütf\0");
+    row.addObject("binary-array", new byte[] { 0, 1, 2, 3, 4 });
+    ByteBuffer binaryBuffer = ByteBuffer.wrap(new byte[] { 5, 6, 7, 8, 9 });
+    row.addObject("binary-bytebuffer", binaryBuffer);
+    row.addObject("null", null);
+    row.addObject("decimal", BigDecimal.valueOf(12345, 3));
+
+    PartialRow expected = getPartialRowWithAllTypes();
+    for (ColumnSchema col : schema.getColumns()) {
+      assertEquals(callGetByName(expected, col.getName(), col.getType()),
+          callGetByName(row, col.getName(), col.getType()));
+    }
+  }
+
   @Test(expected = IllegalArgumentException.class)
   public void testGetNullColumn() {
     PartialRow partialRow = getPartialRowWithAllTypes();
@@ -154,6 +184,10 @@ public class TestPartialRow {
     PartialRow partialRow = getPartialRowWithAllTypes();
     for (ColumnSchema columnSchema : partialRow.getSchema().getColumns()) {
       try {
+        // Skip the null column because `isNull` is not type specific.
+        if ("null".equals(columnSchema.getName())) {
+          continue;
+        }
         callGetByName(partialRow, columnSchema.getName(), getShiftedType(columnSchema.getType()));
         fail("Expected IllegalArgumentException for type: " + columnSchema.getType());
       } catch (IllegalArgumentException ex) {
@@ -400,29 +434,6 @@ public class TestPartialRow {
     return partialRow.getSchema().getColumnIndex(columnName);
   }
 
-  private PartialRow getPartialRowWithAllTypes() {
-    Schema schema = getSchemaWithAllTypes();
-    // Ensure we aren't missing any types
-    assertEquals(13, schema.getColumnCount());
-
-    PartialRow row = schema.newPartialRow();
-    row.addByte("int8", (byte) 42);
-    row.addShort("int16", (short) 43);
-    row.addInt("int32", 44);
-    row.addLong("int64", 45);
-    row.addTimestamp("timestamp", new Timestamp(1234567890));
-    row.addBoolean("bool", true);
-    row.addFloat("float", 52.35F);
-    row.addDouble("double", 53.35);
-    row.addString("string", "fun with ütf\0");
-    row.addBinary("binary-array", new byte[] { 0, 1, 2, 3, 4 });
-    ByteBuffer binaryBuffer = ByteBuffer.wrap(new byte[] { 5, 6, 7, 8, 9 });
-    row.addBinary("binary-bytebuffer", binaryBuffer);
-    row.setNull("null");
-    row.addDecimal("decimal", BigDecimal.valueOf(12345, 3));
-    return row;
-  }
-
   // Shift the type one position to force the wrong type for all types.
   private Type getShiftedType(Type type) {
     int shiftedPosition = (type.ordinal() + 1) % Type.values().length;
@@ -430,6 +441,9 @@ public class TestPartialRow {
   }
 
   private Object callGetByName(PartialRow partialRow, String columnName, Type type) {
+    if (partialRow.isNull(columnName)) {
+      return null;
+    }
     switch (type) {
       case INT8: return partialRow.getByte(columnName);
       case INT16: return partialRow.getShort(columnName);
@@ -448,6 +462,9 @@ public class TestPartialRow {
   }
 
   private Object callGetByIndex(PartialRow partialRow, int columnIndex, Type type) {
+    if (partialRow.isNull(columnIndex)) {
+      return null;
+    }
     switch (type) {
       case INT8: return partialRow.getByte(columnIndex);
       case INT16: return partialRow.getShort(columnIndex);
diff --git a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/ClientTestUtil.java b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/ClientTestUtil.java
index 5e3a37b..ba6842b 100644
--- a/java/kudu-test-utils/src/main/java/org/apache/kudu/test/ClientTestUtil.java
+++ b/java/kudu-test-utils/src/main/java/org/apache/kudu/test/ClientTestUtil.java
@@ -50,11 +50,16 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.junit.Assert.assertEquals;
+
 /**
  * Utilities useful for cluster testing.
  */
@@ -213,6 +218,29 @@ public abstract class ClientTestUtil {
     return new Schema(columns);
   }
 
+  public static PartialRow getPartialRowWithAllTypes() {
+    Schema schema = getSchemaWithAllTypes();
+    // Ensure we aren't missing any types
+    assertEquals(13, schema.getColumnCount());
+
+    PartialRow row = schema.newPartialRow();
+    row.addByte("int8", (byte) 42);
+    row.addShort("int16", (short) 43);
+    row.addInt("int32", 44);
+    row.addLong("int64", 45);
+    row.addTimestamp("timestamp", new Timestamp(1234567890));
+    row.addBoolean("bool", true);
+    row.addFloat("float", 52.35F);
+    row.addDouble("double", 53.35);
+    row.addString("string", "fun with ütf\0");
+    row.addBinary("binary-array", new byte[] { 0, 1, 2, 3, 4 });
+    ByteBuffer binaryBuffer = ByteBuffer.wrap(new byte[] { 5, 6, 7, 8, 9 });
+    row.addBinary("binary-bytebuffer", binaryBuffer);
+    row.setNull("null");
+    row.addDecimal("decimal", BigDecimal.valueOf(12345, 3));
+    return row;
+  }
+
   public static CreateTableOptions getAllTypesCreateTableOptions() {
     return new CreateTableOptions().setRangePartitionColumns(ImmutableList.of("int8"));
   }