[jira] [Commented] (KAFKA-6913) Add primitive numeric converters to Connect

2018-05-29 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-6913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16494454#comment-16494454
 ] 

ASF GitHub Bot commented on KAFKA-6913:
---

ewencp closed pull request #5034: KAFKA-6913: Add Connect converters and header 
converters for short, integer, long, float, and double (WIP)
URL: https://github.com/apache/kafka/pull/5034
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java
 
b/connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java
new file mode 100644
index 000..04019a7a529
--- /dev/null
+++ 
b/connect/api/src/main/java/org/apache/kafka/connect/storage/DoubleConverter.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.storage;
+
+import org.apache.kafka.common.serialization.DoubleDeserializer;
+import org.apache.kafka.common.serialization.DoubleSerializer;
+import org.apache.kafka.connect.data.Schema;
+
+/**
+ * {@link Converter} and {@link HeaderConverter} implementation that only 
supports serializing to and deserializing from double values.
+ * It does support handling nulls. When converting from bytes to Kafka Connect 
format, the converter will always return an
+ * optional FLOAT64 schema.
+ * 
+ * This implementation currently does nothing with the topic names or header 
names.
+ */
+public class DoubleConverter extends NumberConverter {
+
+public DoubleConverter() {
+super("double", Schema.OPTIONAL_FLOAT64_SCHEMA, new 
DoubleSerializer(), new DoubleDeserializer());
+}
+}
diff --git 
a/connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java
 
b/connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java
new file mode 100644
index 000..16bf0e0f93f
--- /dev/null
+++ 
b/connect/api/src/main/java/org/apache/kafka/connect/storage/FloatConverter.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.connect.storage;
+
+import org.apache.kafka.common.serialization.FloatDeserializer;
+import org.apache.kafka.common.serialization.FloatSerializer;
+import org.apache.kafka.connect.data.Schema;
+
+/**
+ * {@link Converter} and {@link HeaderConverter} implementation that only 
supports serializing to and deserializing from float values.
+ * It does support handling nulls. When converting from bytes to Kafka Connect 
format, the converter will always return an
+ * optional FLOAT32 schema.
+ * 
+ * This implementation currently does nothing with the topic names or header 
names.
+ */
+public class FloatConverter extends NumberConverter {
+
+public FloatConverter() {
+super("float", Schema.OPTIONAL_FLOAT32_SCHEMA, new FloatSerializer(), 
new FloatDeserializer());
+}
+}
diff --git 
a/connect/api/src/main/java/org/apache/kafka/connect/storage/IntegerConverter.java
 
b/connect/api/src/main/java/org/apache/kafka/connect/storage/IntegerConverter.java
new file mode 100644
index 000..6f3c78a0a73
--- /dev/null
+++ 

[jira] [Commented] (KAFKA-6913) Add primitive numeric converters to Connect

2018-05-17 Thread Randall Hauch (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16479860#comment-16479860
 ] 

Randall Hauch commented on KAFKA-6913:
--

I created 
[KIP-305|https://cwiki.apache.org/confluence/display/KAFKA/KIP-305%3A+Add+Connect+primitive+number+converters]
 to describe the proposed approach.

See also the [PR|https://github.com/apache/kafka/pull/5034] with the proposed 
changes.

> Add primitive numeric converters to Connect
> ---
>
> Key: KAFKA-6913
> URL: https://issues.apache.org/jira/browse/KAFKA-6913
> Project: Kafka
>  Issue Type: Improvement
>  Components: KafkaConnect
>Affects Versions: 1.1.0
>Reporter: Randall Hauch
>Assignee: Randall Hauch
>Priority: Major
>  Labels: needs-kip
>
> Kafka common includes serdes for long, int, short, float, and double types, 
> but Connect does not have converters for these. They should support null.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-6913) Add primitive numeric converters to Connect

2018-05-17 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16479859#comment-16479859
 ] 

ASF GitHub Bot commented on KAFKA-6913:
---

rhauch opened a new pull request #5034: KAFKA-6913: Add Connect converters and 
header converters for short, integer, long, float, and double (WIP)
URL: https://github.com/apache/kafka/pull/5034
 
 
   *Do not merge until 
[KIP-305](https://cwiki.apache.org/confluence/display/KAFKA/KIP-305%3A+Add+Connect+primitive+number+converters)
 is approved.*
   
   Added converters and header converters for the primitive number types for 
which Kafka already had serializers and deserializers. All extend a common base 
class, `NumberConverter`, that encapsulates most of the shared functionality. 
Unit tests were added to check the basic functionality.
   
   These classes are not used by any other Connect code, and must be explicitly 
used in Connect workers and connectors.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Add primitive numeric converters to Connect
> ---
>
> Key: KAFKA-6913
> URL: https://issues.apache.org/jira/browse/KAFKA-6913
> Project: Kafka
>  Issue Type: Improvement
>  Components: KafkaConnect
>Affects Versions: 1.1.0
>Reporter: Randall Hauch
>Assignee: Randall Hauch
>Priority: Major
>  Labels: needs-kip
>
> Kafka common includes serdes for long, int, short, float, and double types, 
> but Connect does not have converters for these. They should support null.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)