kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ewe...@apache.org
Subject kafka git commit: KAFKA-5548: Extended validation for SchemaBuilder methods.
Date Sat, 01 Jul 2017 09:47:01 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 9238aeaa2 -> ab8e9d175


KAFKA-5548: Extended validation for SchemaBuilder methods.

More input validation for SchemaBuilder methods.

Author: Jeremy Custenborder <jcustenborder@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #3474 from jcustenborder/KAFKA-5548


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

Branch: refs/heads/trunk
Commit: ab8e9d1755e9527ca76e994dd59a21cc94550312
Parents: 9238aea
Author: Jeremy Custenborder <jcustenborder@gmail.com>
Authored: Sat Jul 1 02:46:55 2017 -0700
Committer: Ewen Cheslack-Postava <me@ewencp.org>
Committed: Sat Jul 1 02:46:55 2017 -0700

----------------------------------------------------------------------
 .../kafka/connect/data/SchemaBuilder.java       | 10 ++++++
 .../kafka/connect/data/SchemaBuilderTest.java   | 32 ++++++++++++++++++++
 2 files changed, 42 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/ab8e9d17/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java
index 058660e..e0109bb 100644
--- a/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java
+++ b/connect/api/src/main/java/org/apache/kafka/connect/data/SchemaBuilder.java
@@ -319,6 +319,10 @@ public class SchemaBuilder implements Schema {
     public SchemaBuilder field(String fieldName, Schema fieldSchema) {
         if (type != Type.STRUCT)
             throw new SchemaBuilderException("Cannot create fields on type " + type);
+        if (null == fieldName || fieldName.isEmpty())
+            throw new SchemaBuilderException("fieldName cannot be null.");
+        if (null == fieldSchema)
+            throw new SchemaBuilderException("fieldSchema for field " + fieldName + " cannot
be null.");
         int fieldIndex = fields.size();
         if (fields.containsKey(fieldName))
             throw new SchemaBuilderException("Cannot create field because of field name duplication
" + fieldName);
@@ -351,6 +355,8 @@ public class SchemaBuilder implements Schema {
      * @return a new {@link Schema.Type#ARRAY} SchemaBuilder
      */
     public static SchemaBuilder array(Schema valueSchema) {
+        if (null == valueSchema)
+            throw new SchemaBuilderException("valueSchema cannot be null.");
         SchemaBuilder builder = new SchemaBuilder(Type.ARRAY);
         builder.valueSchema = valueSchema;
         return builder;
@@ -362,6 +368,10 @@ public class SchemaBuilder implements Schema {
      * @return a new {@link Schema.Type#MAP} SchemaBuilder
      */
     public static SchemaBuilder map(Schema keySchema, Schema valueSchema) {
+        if (null == keySchema)
+            throw new SchemaBuilderException("keySchema cannot be null.");
+        if (null == valueSchema)
+            throw new SchemaBuilderException("valueSchema cannot be null.");
         SchemaBuilder builder = new SchemaBuilder(Type.MAP);
         builder.keySchema = keySchema;
         builder.valueSchema = valueSchema;

http://git-wip-us.apache.org/repos/asf/kafka/blob/ab8e9d17/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java
----------------------------------------------------------------------
diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java
b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java
index 0dba0ad..886a5e7 100644
--- a/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java
+++ b/connect/api/src/test/java/org/apache/kafka/connect/data/SchemaBuilderTest.java
@@ -323,6 +323,38 @@ public class SchemaBuilderTest {
         schemaBuilder.version(456);
     }
 
+    @Test(expected = SchemaBuilderException.class)
+    public void testFieldNameNull() {
+        Schema schema = SchemaBuilder.struct()
+            .field(null, Schema.STRING_SCHEMA)
+            .build();
+    }
+
+    @Test(expected = SchemaBuilderException.class)
+    public void testFieldSchemaNull() {
+        Schema schema = SchemaBuilder.struct()
+            .field("fieldName", null)
+            .build();
+    }
+
+    @Test(expected = SchemaBuilderException.class)
+    public void testArraySchemaNull() {
+        Schema schema = SchemaBuilder.array(null)
+            .build();
+    }
+
+    @Test(expected = SchemaBuilderException.class)
+    public void testMapKeySchemaNull() {
+        Schema schema = SchemaBuilder.map(null, Schema.STRING_SCHEMA)
+            .build();
+    }
+
+    @Test(expected = SchemaBuilderException.class)
+    public void testMapValueSchemaNull() {
+        Schema schema = SchemaBuilder.map(Schema.STRING_SCHEMA, null)
+            .build();
+    }
+
     private void assertTypeAndDefault(Schema schema, Schema.Type type, boolean optional,
Object defaultValue) {
         assertEquals(type, schema.type());
         assertEquals(optional, schema.isOptional());


Mime
View raw message