kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gwens...@apache.org
Subject kafka git commit: KAFKA-4929: Transformation Key/Value type references should be to class name(), not canonicalName()
Date Wed, 22 Mar 2017 21:02:17 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 666beecec -> 763ea5aad


KAFKA-4929: Transformation Key/Value type references should be to class name(), not canonicalName()

Changing getCanonicalName() references to getName() so that docs update with "$" instead of
".".  Also added a connect-plugin-discovery.sh CLI to list all of the transformations available.

Author: Bruce Szalwinski <bruce.szalwinski@cdk.com>

Reviewers: Gwen Shapira

Closes #2720 from bruce-szalwinski/transforms and squashes the following commits:

ec3b5b9 [Bruce Szalwinski] remove connect-plugin-discovery.  will submit in a different PR
eba0af7 [Bruce Szalwinski] Key / Value transformations are static nested classes and so are
referenced using OuterClass$Key and OuterClass$Value.


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

Branch: refs/heads/trunk
Commit: 763ea5aad191b09ed3deefaa12ce7b9925ebe7dc
Parents: 666beec
Author: Bruce Szalwinski <bruce.szalwinski@cdk.com>
Authored: Wed Mar 22 16:02:09 2017 -0500
Committer: Gwen Shapira <cshapi@gmail.com>
Committed: Wed Mar 22 16:02:09 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/kafka/connect/runtime/PluginDiscovery.java   | 2 +-
 .../java/org/apache/kafka/connect/transforms/ExtractField.java   | 4 ++--
 .../java/org/apache/kafka/connect/transforms/HoistField.java     | 4 ++--
 .../java/org/apache/kafka/connect/transforms/InsertField.java    | 4 ++--
 .../main/java/org/apache/kafka/connect/transforms/MaskField.java | 4 ++--
 .../java/org/apache/kafka/connect/transforms/ReplaceField.java   | 4 +++-
 .../org/apache/kafka/connect/transforms/SetSchemaMetadata.java   | 4 ++--
 7 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
index be55daf..d31ce6d 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java
@@ -106,7 +106,7 @@ public class PluginDiscovery {
         Collections.sort(transformationPlugins, new Comparator<Class<? extends Transformation>>()
{
             @Override
             public int compare(Class<? extends Transformation> a, Class<? extends
Transformation> b) {
-                return a.getCanonicalName().compareTo(b.getCanonicalName());
+                return a.getName().compareTo(b.getName());
             }
         });
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java
index 36373eb..bb4b539 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java
@@ -31,8 +31,8 @@ public abstract class ExtractField<R extends ConnectRecord<R>>
implements Transf
 
     public static final String OVERVIEW_DOC =
             "Extract the specified field from a Struct when schema present, or a Map in the
case of schemaless data."
-                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getCanonicalName() + "</code>) "
-                    + "or value (<code>" + Value.class.getCanonicalName() + "</code>).";
+                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getName() + "</code>) "
+                    + "or value (<code>" + Value.class.getName() + "</code>).";
 
     private static final String FIELD_CONFIG = "field";
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java
index 1035e97..a852c79 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java
@@ -33,8 +33,8 @@ public abstract class HoistField<R extends ConnectRecord<R>>
implements Transfor
 
     public static final String OVERVIEW_DOC =
             "Wrap data using the specified field name in a Struct when schema present, or
a Map in the case of schemaless data."
-                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getCanonicalName() + "</code>) "
-                    + "or value (<code>" + Value.class.getCanonicalName() + "</code>).";
+                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getName() + "</code>) "
+                    + "or value (<code>" + Value.class.getName() + "</code>).";
 
     private static final String FIELD_CONFIG = "field";
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
index 441129a..5e472a9 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java
@@ -42,8 +42,8 @@ public abstract class InsertField<R extends ConnectRecord<R>>
implements Transfo
 
     public static final String OVERVIEW_DOC =
             "Insert field(s) using attributes from the record metadata or a configured static
value."
-                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getCanonicalName() + "</code>) "
-                    + "or value (<code>" + Value.class.getCanonicalName() + "</code>).";
+                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getName() + "</code>) "
+                    + "or value (<code>" + Value.class.getName() + "</code>).";
 
     private interface ConfigName {
         String TOPIC_FIELD = "topic.field";

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java
index f9946a0..050c549 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java
@@ -42,8 +42,8 @@ public abstract class MaskField<R extends ConnectRecord<R>>
implements Transform
 
     public static final String OVERVIEW_DOC =
             "Mask specified fields with a valid null value for the field type (i.e. 0, false,
empty string, and so on)."
-                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getCanonicalName() + "</code>) "
-                    + "or value (<code>" + Value.class.getCanonicalName() + "</code>).";
+                    + "<p/>Use the concrete transformation type designed for the record
key (<code>" + Key.class.getName() + "</code>) "
+                    + "or value (<code>" + Value.class.getName() + "</code>).";
 
     private static final String FIELDS_CONFIG = "fields";
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
index 8bdc153..ee08945 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java
@@ -39,7 +39,9 @@ import static org.apache.kafka.connect.transforms.util.Requirements.requireStruc
 
 public abstract class ReplaceField<R extends ConnectRecord<R>> implements Transformation<R>
{
 
-    public static final String OVERVIEW_DOC = "Filter or rename fields.";
+    public static final String OVERVIEW_DOC = "Filter or rename fields."
+            + "<p/>Use the concrete transformation type designed for the record key
(<code>" + Key.class.getName() + "</code>) "
+            + "or value (<code>" + Value.class.getName() + "</code>).";
 
     interface ConfigName {
         String BLACKLIST = "blacklist";

http://git-wip-us.apache.org/repos/asf/kafka/blob/763ea5aa/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
----------------------------------------------------------------------
diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
index d34a430..bb581de 100644
--- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
+++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java
@@ -30,8 +30,8 @@ import static org.apache.kafka.connect.transforms.util.Requirements.requireSchem
 public abstract class SetSchemaMetadata<R extends ConnectRecord<R>> implements
Transformation<R> {
 
     public static final String OVERVIEW_DOC =
-            "Set the schema name, version or both on the record's key (<code>" + Key.class.getCanonicalName()
+ "</code>)"
-                    + " or value (<code>" + Value.class.getCanonicalName() + "</code>)
schema.";
+            "Set the schema name, version or both on the record's key (<code>" + Key.class.getName()
+ "</code>)"
+                    + " or value (<code>" + Value.class.getName() + "</code>)
schema.";
 
     private interface ConfigName {
         String SCHEMA_NAME = "schema.name";


Mime
View raw message