[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2021-04-27 Thread Flink Jira Bot (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17334115#comment-17334115
 ] 

Flink Jira Bot commented on FLINK-10122:


This issue was marked "stale-assigned" and has not received an update in 7 
days. It is now automatically unassigned. If you are still working on it, you 
can assign it to yourself again. Please also give an update about the status of 
the work.

> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Connectors / Kafka
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available, stale-assigned
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2021-04-16 Thread Flink Jira Bot (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17323489#comment-17323489
 ] 

Flink Jira Bot commented on FLINK-10122:


This issue is assigned but has not received an update in 7 days so it has been 
labeled "stale-assigned". If you are still working on the issue, please give an 
update and remove the label. If you are no longer working on the issue, please 
unassign so someone else may work on it. In 7 days the issue will be 
automatically unassigned.

> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Connectors / Kafka
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available, stale-assigned
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-09-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609077#comment-16609077
 ] 

ASF GitHub Bot commented on FLINK-10122:


StefanRRichter closed pull request #6537: [FLINK-10122] KafkaConsumer should 
use partitionable state over union state if partition discovery is not active
URL: https://github.com/apache/flink/pull/6537
 
 
   

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/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 
b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
index cfb5b6d510d..3857a968dd5 100644
--- 
a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ 
b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -105,8 +105,11 @@
/** Configuration key to define the consumer's partition discovery 
interval, in milliseconds. */
public static final String KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS = 
"flink.partition-discovery.interval-millis";
 
+   /** For backwards compatibility. */
+   private static final String OLD_OFFSETS_STATE_NAME = 
"topic-partition-offset-states";
+
/** State name of the consumer's partition offset states. */
-   private static final String OFFSETS_STATE_NAME = 
"topic-partition-offset-states";
+   private static final String OFFSETS_STATE_NAME = 
"kafka-consumer-offsets";
 
// 

//  configuration state, set on the client relevant for all subtasks
@@ -180,13 +183,7 @@
private transient volatile TreeMap 
restoredState;
 
/** Accessor for state in the operator state backend. */
-   private transient ListState> 
unionOffsetStates;
-
-   /**
-* Flag indicating whether the consumer is restored from older state 
written with Flink 1.1 or 1.2.
-* When the current run is restored from older state, partition 
discovery is disabled.
-*/
-   private boolean restoredFromOldState;
+   private transient ListState> 
offsetsState;
 
/** Discovery loop, executed in a separate thread. */
private transient volatile Thread discoveryLoopThread;
@@ -480,7 +477,7 @@ public void open(Configuration configuration) throws 
Exception {
}
 
for (Map.Entry 
restoredStateEntry : restoredState.entrySet()) {
-   if (!restoredFromOldState) {
+   if (discoveryIntervalMillis != 
PARTITION_DISCOVERY_DISABLED) {
// seed the partition discoverer with 
the union state while filtering out
// restored partitions that should not 
be subscribed by this subtask
if (KafkaTopicPartitionAssigner.assign(
@@ -489,8 +486,7 @@ public void open(Configuration configuration) throws 
Exception {

subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), 
restoredStateEntry.getValue());
}
} else {
-   // when restoring from older 1.1 / 1.2 
state, the restored state would not be the union state;
-   // in this case, just use the restored 
state as the subscribed partitions
+   // just restore from assigned partitions

subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), 
restoredStateEntry.getValue());
}
}
@@ -783,30 +779,26 @@ public final void 
initializeState(FunctionInitializationContext context) throws
 
OperatorStateStore stateStore = context.getOperatorStateStore();
 
-   ListState> 
oldRoundRobinListState =
-   
stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
+   final TypeInformation> 
offsetStateTypeInfo =
+   TypeInformation.of(new 
TypeHint>() {});
 
-   this.unionOffsetStates = stateStore.getUnionListState(new 
ListStateDescriptor<>(
-   OFFSETS_STATE_NAME,
-   TypeInformation.of(

[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

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


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16583549#comment-16583549
 ] 

ASF GitHub Bot commented on FLINK-10122:


StefanRRichter commented on issue #6537: [FLINK-10122] KafkaConsumer should use 
partitionable state over union state if partition discovery is not active
URL: https://github.com/apache/flink/pull/6537#issuecomment-413787110
 
 
   Thanks @tzulitai ! I was aware that this will break the behavior for 
partition discovery. However, the current implementation was already broken for 
user at large scale, as pointed out in the description. This PR was intended as 
a quick solution for this case. I think that we can have better non-breaking 
solutions in the future like splitting the source into two operators or a 
different state partitioning scheme. I think that we can close the PR and go 
for the long term solution in official releases. Nevertheless I think that we 
should cherry-pick two parts of this PR into releases, the hotfix to improve 
memory utilization and the option to remove operator state (or - even better - 
states in general).


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16583386#comment-16583386
 ] 

ASF GitHub Bot commented on FLINK-10122:


tzulitai commented on a change in pull request #6537: [FLINK-10122] 
KafkaConsumer should use partitionable state over union state if partition 
discovery is not active
URL: https://github.com/apache/flink/pull/6537#discussion_r210808619
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##
 @@ -480,7 +477,7 @@ public void open(Configuration configuration) throws 
Exception {
}
 
for (Map.Entry 
restoredStateEntry : restoredState.entrySet()) {
-   if (!restoredFromOldState) {
+   if (discoveryIntervalMillis != 
PARTITION_DISCOVERY_DISABLED) {
 
 Review comment:
   This, therefore, is a bit confusing.
   
   If we decide to change it so that whether or not partition discovery is 
enabled is determined at the very beginning and cannot be changed from them on 
(even on restarts), then there shouldn't be a case to check 
`discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED` here.


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16583385#comment-16583385
 ] 

ASF GitHub Bot commented on FLINK-10122:


tzulitai commented on a change in pull request #6537: [FLINK-10122] 
KafkaConsumer should use partitionable state over union state if partition 
discovery is not active
URL: https://github.com/apache/flink/pull/6537#discussion_r210808465
 
 

 ##
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##
 @@ -783,30 +779,26 @@ public final void 
initializeState(FunctionInitializationContext context) throws
 
OperatorStateStore stateStore = context.getOperatorStateStore();
 
-   ListState> 
oldRoundRobinListState =
-   
stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
+   final TypeInformation> 
offsetStateTypeInfo =
+   TypeInformation.of(new 
TypeHint>() {});
 
-   this.unionOffsetStates = stateStore.getUnionListState(new 
ListStateDescriptor<>(
-   OFFSETS_STATE_NAME,
-   TypeInformation.of(new 
TypeHint>() {})));
+   ListStateDescriptor> 
offsetStateDescriptor =
+   new ListStateDescriptor<>(OFFSETS_STATE_NAME, 
offsetStateTypeInfo);
 
-   if (context.isRestored() && !restoredFromOldState) {
-   restoredState = new TreeMap<>(new 
KafkaTopicPartition.Comparator());
+   this.offsetsState =
+   discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED 
?
+   
stateStore.getUnionListState(offsetStateDescriptor) : 
stateStore.getListState(offsetStateDescriptor);
 
 Review comment:
   I think this is a "behaviour break".
   Before, the user was free to switch between enabling / disabling partition 
discovery on each restart for the job.
   Now, with this change, we can't really do that, because changing the 
distribution scheme of the same registered state isn't allowed.


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16576396#comment-16576396
 ] 

ASF GitHub Bot commented on FLINK-10122:


StefanRRichter commented on a change in pull request #6537: [FLINK-10122] 
KafkaConsumer should use partitionable state over union state if partition 
discovery is not active
URL: https://github.com/apache/flink/pull/6537#discussion_r209291672
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
 ##
 @@ -196,6 +196,20 @@ public void removeOperatorState(String name) {
}
}
 
+   public void deleteBroadCastState(String name) {
 
 Review comment:
   Argh...will remove the second one ;)


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16576394#comment-16576394
 ] 

ASF GitHub Bot commented on FLINK-10122:


aljoscha commented on a change in pull request #6537: [FLINK-10122] 
KafkaConsumer should use partitionable state over union state if partition 
discovery is not active
URL: https://github.com/apache/flink/pull/6537#discussion_r209291065
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
 ##
 @@ -196,6 +196,20 @@ public void removeOperatorState(String name) {
}
}
 
+   public void deleteBroadCastState(String name) {
 
 Review comment:
   I think you added these methods twice, once under `deleteBroadcastState` and 
once under `deleteBroadCastState`.


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16576171#comment-16576171
 ] 

ASF GitHub Bot commented on FLINK-10122:


StefanRRichter commented on issue #6537: [FLINK-10122] KafkaConsumer should use 
partitionable state over union state if partition discovery is not active
URL: https://github.com/apache/flink/pull/6537#issuecomment-412061048
 
 
   CC @tzulitai or @aljoscha 


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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


[jira] [Commented] (FLINK-10122) KafkaConsumer should use partitionable state over union state if partition discovery is not active

2018-08-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16576169#comment-16576169
 ] 

ASF GitHub Bot commented on FLINK-10122:


StefanRRichter opened a new pull request #6537: [FLINK-10122] KafkaConsumer 
should use partitionable state over union state if partition discovery is not 
active
URL: https://github.com/apache/flink/pull/6537
 
 
   ## What is the purpose of the change
   
   KafkaConsumer store its offsets state always as union state. I think this is 
only required in the case that partition discovery is active. For jobs with a 
very high parallelism, the union state can lead to prohibitively expensive 
deployments. For example, a job with 2000 source and a total of 10MB 
checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 20GB 
of state. With partitionable state, it would have to ship ~10MB.
   
   For now, I would suggest to go back to partitionable state in case that 
partition discovery is not active. In the long run, I have some ideas for more 
efficient partitioning schemes that would also work for active discovery.
   
   As an additional step to support backwards compatibility and avoid state 
blowup from the compatibility state, I also implemented FLINK-10121, so that 
operator state can be removed from the backend.
   
   Included also a hotfix that makes state assignment on the JM more memory 
friendly by trying to allocate all collections with good sizes.
   
   ## Verifying this change
   
   This change is already covered by existing tests.
   
   ## Does this pull request potentially affect one of the following parts:
   
 - Dependencies (does it add or upgrade a dependency):  (no)
 - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes)
 - The serializers: (no)
 - The runtime per-record code paths (performance sensitive): (no)
 - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
 - The S3 file system connector: (no)
   
   ## Documentation
   
 - Does this pull request introduce a new feature? (no)
 - If yes, how is the feature documented? (not applicable)
   


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


> KafkaConsumer should use partitionable state over union state if partition 
> discovery is not active
> --
>
> Key: FLINK-10122
> URL: https://issues.apache.org/jira/browse/FLINK-10122
> Project: Flink
>  Issue Type: Improvement
>  Components: Kafka Connector
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.7.0
>
>
> KafkaConsumer store its offsets state always as union state. I think this is 
> only required in the case that partition discovery is active. For jobs with a 
> very high parallelism, the union state can lead to prohibitively expensive 
> deployments. For example, a job with 2000 source and a total of 10MB 
> checkpointed union state offsets state would have to ship ~ 2000 x 10MB = 
> 20GB of state. With partitionable state, it would have to ship ~10MB.
> For now, I would suggest to go back to partitionable state in case that 
> partition discovery is not active. In the long run, I have some ideas for 
> more efficient partitioning schemes that would also work for active discovery.



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