[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-19 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r257965533
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   The change for `KafkaDeserialisationSchema` might be simple if we do it like 
this:
   ```
   // new API, that we could now use everywhere in Kafka code instead of 
KeyedDeserializationSchema
   interface KafkaDeserialisationSchema {
 T deserialize(ConsumerRecord record);
   }
   
   // old API, stays almost the same as we already have it in the PR and also 
supports the new API
   @deprecated
   interface KeyedDeserializationSchema extends 
KafkaDeserialisationSchema {
 T deserialize(byte[] messageKey, byte[] message, String topic, int 
partition, long offset);
 default T deserialize(ConsumerRecord record) {
   return deserialize(record.key(), record.value(), record.topic(), 
record.partition(), record.offset());
 }
   }
   ```
   Do you think it is quick enough change?
   
   Ok, I see your point with tests regarding the splitting. The thing is that 
we are interested in merging this PR this week before feature freeze for 1.8 
release (planned this Friday). Let's see then how it goes and how much we can 
manage. Do you have time this week to work on this?


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-18 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r257804466
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   Thanks, I will have a look. 
   
   Do you also plan to introduce the similar `KafkaDeserialisationSchema` to 
covert `ConsumerRecord` into object to have symmetrical API?
   
   Also, it would be nice to split this big PR into two PRs with separate 
consumer and producer refactoring. The consumer part seems to be more clear and 
popular as a feature, we could try to merge it first.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-18 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r257603799
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   I would keep it in the constructor because it will be the same for each 
`serialize` method. We could consider having overloaded 
`PartitionInfo.getPartitions()` method but this would also mean some implicit 
semantics of `PartitionInfo` having `defaultTopic` from `FlinkKafkaProducer`.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-11 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r255449688
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   True, it should be `PartitionInfo.getPartitions(topic)` and the adaptor has 
to have the `defaultTopic`.
   Maybe, we could also pack `topicPartitionsMap` and `getPartitionsByTopic` 
logic into `PartitionInfo` which would be one object belonging to 
`FlinkKafkaProducer`.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-11 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r255449688
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   True, it should be `PartitionInfo.getPartitions(topic)` and the adaptor has 
to have the `defaultTopic`.
   Maybe, we could pack `topicPartitionsMap` and `getPartitionsByTopic` logic 
into `PartitionInfo` which would be one object belonging to 
`FlinkKafkaProducer`.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-10 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r255333491
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   Yes, I would try with 2 adaptors and accept the migration risk. API w/o open 
method looks simpler. It can be extended later anyways if we discover use cases 
for it.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-06 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r254226128
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   Let's say the new serialization schema looks like this:
   ```
   interface KafkaSerializationSchema {
 ProducerRecord serialize(T element, @Nullable Long timestamp, 
PartitionInfo partitionInfo);
   }
   class PartitionInfo {
 private final int parallelInstanceId, int parallelInstances, int[] 
partitions);
 // 
   }
   ```
   This way it should preserve previous Partitioner API capabilities.
   
   Regarding the multiple adaptors, from what I see, the difference is that the 
older connectors do not support timestamp in `ProducerRecord`. Could we have 
one adaptor in base module which calls `ProducerRecord` constructor without 
timestamp if `KafkaSerializationSchema.serialize` is called with `timestamp = 
null`? Producers, which do not supported timestamp, could call new schema with 
`timestamp = null`, including base one (although `ProducerRecord` has 
`timestamp` in base).
   
   Regarding the open method, could adaptor or partitioner still be opened the 
similar way as it happens now to current partitioner? I agree that the new 
schema would look weird with the open method.
   
   `PartitionInfo` could be populated in a similar way as `int[] partitions` 
are cached now in `FlinkKafkaProducer.topicPartitionsMap` in 
`FlinkKafkaProducer.invoke`, just `parallelInstanceId` and `parallelInstances` 
are added.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-02-01 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r253055987
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   @alexeyt820 
   Ideally we could deprecate the partitioner in the producer constructer as 
well because ProducerRecord already contains partition which user can assign 
for the record.
   
   Instead of deprecating the methods in (de)ser schema interfaces we could 
deprecate them fully and introduce new `Kafka(De)SerializationSchema` 
interfaces which work with Kafka `Consumer/ProducerRecord` classes.
   
   We also introduce adaptors from older schemas to the newer ones. The 
producer/consumer constructors, which currently accept older schemas, will use 
adaptors to create newer schemas. The actual code should also work with newer 
schemas only.
   
   The serialization schema adaptor could optionally take the 
`FlinkKafkaPartitioner` to populate the partition in similar way as now it 
happens directly in producer.
   
   What do you think?


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-24 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r250110054
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   I would even suggest to rewrap Kafka Header class into Flink class here, 
because it should be more extensible if Kafka decides to extend its Header in 
future.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-23 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r250110054
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchema.java
 ##
 @@ -55,4 +57,13 @@
 * @return null or the target topic
 */
String getTargetTopic(T element);
+
+   /**
+*
+* @param element The incoming element to be serialized
+* @return collection of headers (maybe empty)
+*/
+   default Iterable> headers(T element) {
 
 Review comment:
   I would even suggest to rewrap Kafka Header class into Flink class here, 
because it should be more extensible if Kafka decides to extend its Header in 
future. But if it is possible to accept just Kafka `ConsumerRecord` directly 
for all versions then we do not need any `Record` wrapping at all.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-22 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249829200
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
 ##
 @@ -370,8 +370,8 @@ else if (partitionsRemoved) {

keyPayload.get(keyBytes);
}
 
-   final T value = 
deserializer.deserialize(keyBytes, valueBytes,
-   
currentPartition.getTopic(), currentPartition.getPartition(), offset);
+   final T value = 
deserializer.deserialize(
 
 Review comment:
   I would be ok with having `Exception` in the signature of `deserialize`.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249512219
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
 ##
 @@ -370,8 +370,8 @@ else if (partitionsRemoved) {

keyPayload.get(keyBytes);
}
 
-   final T value = 
deserializer.deserialize(keyBytes, valueBytes,
-   
currentPartition.getTopic(), currentPartition.getPartition(), offset);
+   final T value = 
deserializer.deserialize(
 
 Review comment:
   Here we could try:
   ```
   ConsumerRecord consumerRecord = 
  new ConsumerRecord<>(currentPartition.getTopic(), 
currentPartition.getPartition(), 
 keyBytes, valueBytes, offset);
   final T value = deserializer.deserialize(consumerRecord);
   ```


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249368384
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka011ConsumerRecord.java
 ##
 @@ -0,0 +1,57 @@
+/*
+ * 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.flink.streaming.connectors.kafka.internal;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Function;
+import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.AbstractMap;
+import java.util.Map;
+
+/**
+ * Extends base Kafka09ConsumerRecord to provide access to Kafka headers.
 
 Review comment:
   could you put into `{@link Kafka09ConsumerRecord}`


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249478973
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchema.java
 ##
 @@ -32,6 +34,49 @@
  */
 @PublicEvolving
 public interface KeyedDeserializationSchema extends Serializable, 
ResultTypeQueryable {
+   /**
+* Kafka record to be deserialized.
+* Record consists of key,value pair, topic name, partition offset, 
headers and a timestamp (if available)
+*/
+   interface Record {
 
 Review comment:
   kafka-clients 0.8 actually has `ConsumerRecord`, just `SimpleConsumerThread` 
does not use it but it seems to be possible to manually wrap `MessageAndOffset` 
into that `ConsumerRecord`. I would give it a try. It seems to be simpler 
option at the moment and would eliminate currently introduced inheritance for 
the sake of wrapping.
   
   Not sure, though, how big the risk is that the Kafka API changes again. The 
Flink wrapping, as now, seems to be a safer option but it also adds some 
performance overhead.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249510977
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##
 @@ -82,93 +84,110 @@
  */
 @Internal
 public abstract class FlinkKafkaConsumerBase extends 
RichParallelSourceFunction implements
-   CheckpointListener,
-   ResultTypeQueryable,
-   CheckpointedFunction {
+   CheckpointListener,
 
 Review comment:
   This class contains a lot of unrelated changes, which makes it more 
difficult to review. I would suggest to have either a follow-up PR for them or 
at least put them into a separate commit.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249482757
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka011ConsumerRecord.java
 ##
 @@ -0,0 +1,57 @@
+/*
+ * 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.flink.streaming.connectors.kafka.internal;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Function;
+import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.AbstractMap;
+import java.util.Map;
+
+/**
+ * Extends base Kafka09ConsumerRecord to provide access to Kafka headers.
+ */
+class Kafka011ConsumerRecord extends Kafka09ConsumerRecord {
+   /**
+* Wraps {@link Header} as Map.Entry.
+*/
+   private static final Function> 
HEADER_TO_MAP_ENTRY_FUNCTION =
+   new Function>() {
+   @Nonnull
+   @Override
+   public Map.Entry apply(@Nullable Header 
header) {
+   return new 
AbstractMap.SimpleImmutableEntry<>(header.key(), header.value());
+   }
+   };
+
+   Kafka011ConsumerRecord(ConsumerRecord consumerRecord) {
+   super(consumerRecord);
+   }
+
+   @Override
+   public Iterable> headers() {
+   return Iterables.transform(consumerRecord.headers(), 
HEADER_TO_MAP_ENTRY_FUNCTION);
 
 Review comment:
   Could we avoid relying on non-standard java libraries like guava?
   The record/headers processing is also on the time critical path of per 
record latency.
   I would suggest to implement our own Iterable wrapper which does only this 
header wrapping.


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249367559
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
 ##
 @@ -641,6 +641,9 @@ public void invoke(KafkaTransactionState transaction, IN 
next, Context context)
} else {
record = new ProducerRecord<>(targetTopic, null, 
timestamp, serializedKey, serializedValue);
}
+   for (Map.Entry header: schema.headers(next)) {
 
 Review comment:
   space before `:`


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


With regards,
Apache Git Services


[GitHub] azagrebin commented on a change in pull request #6615: [FLINK-8354] [flink-connectors] Add ability to access and provider Kafka headers

2019-01-21 Thread GitBox
azagrebin commented on a change in pull request #6615: [FLINK-8354] 
[flink-connectors] Add ability to access and provider Kafka headers
URL: https://github.com/apache/flink/pull/6615#discussion_r249114818
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##
 @@ -251,6 +269,18 @@ public FlinkKafkaConsumerBase(
//  Configuration
// 

 
+   /**
+* Make sure that auto commit is disabled when our offset commit mode 
is ON_CHECKPOINTS.
+* This overwrites whatever setting the user configured in the 
properties.
+* @param properties - Kafka configuration properties to be adjusted
+* @param offsetCommitMode offset commit mode
+*/
+   protected static void adjustAutoCommitConfig(Properties properties, 
OffsetCommitMode offsetCommitMode) {
 
 Review comment:
   The method can be package-private.


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


With regards,
Apache Git Services