Browse Source
- remove type specific methods from Task interface - add generics to preserve task type - add sub classes for different task types Author: Matthias J. Sax <matthias@confluent.io> Reviewers: Bill Bejeck <bill@confluent.io>, Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com> Closes #4129 from mjsax/kafka-6115-taskManager-should-be-type-awarepull/4148/merge
Matthias J. Sax
7 years ago
committed by
Guozhang Wang
16 changed files with 278 additions and 304 deletions
@ -0,0 +1,27 @@
@@ -0,0 +1,27 @@
|
||||
/* |
||||
* 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.streams.processor.internals; |
||||
|
||||
import org.apache.kafka.common.utils.LogContext; |
||||
|
||||
class AssignedStandbyTasks extends AssignedTasks<StandbyTask> { |
||||
|
||||
AssignedStandbyTasks(final LogContext logContext) { |
||||
super(logContext, "standby task"); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,128 @@
@@ -0,0 +1,128 @@
|
||||
/* |
||||
* 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.streams.processor.internals; |
||||
|
||||
import org.apache.kafka.common.KafkaException; |
||||
import org.apache.kafka.common.TopicPartition; |
||||
import org.apache.kafka.common.utils.LogContext; |
||||
import org.apache.kafka.streams.errors.TaskMigratedException; |
||||
import org.apache.kafka.streams.processor.TaskId; |
||||
import org.slf4j.Logger; |
||||
|
||||
import java.util.Iterator; |
||||
import java.util.Map; |
||||
|
||||
class AssignedStreamsTasks extends AssignedTasks<StreamTask> implements RestoringTasks { |
||||
private final Logger log; |
||||
private final TaskAction<StreamTask> maybeCommitAction; |
||||
private int committed = 0; |
||||
|
||||
AssignedStreamsTasks(final LogContext logContext) { |
||||
super(logContext, "stream task"); |
||||
|
||||
this.log = logContext.logger(getClass()); |
||||
|
||||
maybeCommitAction = new TaskAction<StreamTask>() { |
||||
@Override |
||||
public String name() { |
||||
return "maybeCommit"; |
||||
} |
||||
|
||||
@Override |
||||
public void apply(final StreamTask task) { |
||||
if (task.commitNeeded()) { |
||||
committed++; |
||||
task.commit(); |
||||
log.debug("Committed active task {} per user request in", task.id()); |
||||
} |
||||
} |
||||
}; |
||||
} |
||||
|
||||
@Override |
||||
public StreamTask restoringTaskFor(final TopicPartition partition) { |
||||
return restoringByPartition.get(partition); |
||||
} |
||||
|
||||
/** |
||||
* @throws TaskMigratedException if committing offsets failed (non-EOS) |
||||
* or if the task producer got fenced (EOS) |
||||
*/ |
||||
int maybeCommit() { |
||||
committed = 0; |
||||
applyToRunningTasks(maybeCommitAction); |
||||
return committed; |
||||
} |
||||
|
||||
/** |
||||
* @throws TaskMigratedException if the task producer got fenced (EOS only) |
||||
*/ |
||||
int process() { |
||||
int processed = 0; |
||||
final Iterator<Map.Entry<TaskId, StreamTask>> it = running.entrySet().iterator(); |
||||
while (it.hasNext()) { |
||||
final StreamTask task = it.next().getValue(); |
||||
try { |
||||
if (task.process()) { |
||||
processed++; |
||||
} |
||||
} catch (final TaskMigratedException e) { |
||||
final RuntimeException fatalException = closeZombieTask(task); |
||||
if (fatalException != null) { |
||||
throw fatalException; |
||||
} |
||||
it.remove(); |
||||
throw e; |
||||
} catch (final RuntimeException e) { |
||||
log.error("Failed to process stream task {} due to the following error:", task.id(), e); |
||||
throw e; |
||||
} |
||||
} |
||||
return processed; |
||||
} |
||||
|
||||
/** |
||||
* @throws TaskMigratedException if the task producer got fenced (EOS only) |
||||
*/ |
||||
int punctuate() { |
||||
int punctuated = 0; |
||||
final Iterator<Map.Entry<TaskId, StreamTask>> it = running.entrySet().iterator(); |
||||
while (it.hasNext()) { |
||||
final StreamTask task = it.next().getValue(); |
||||
try { |
||||
if (task.maybePunctuateStreamTime()) { |
||||
punctuated++; |
||||
} |
||||
if (task.maybePunctuateSystemTime()) { |
||||
punctuated++; |
||||
} |
||||
} catch (final TaskMigratedException e) { |
||||
final RuntimeException fatalException = closeZombieTask(task); |
||||
if (fatalException != null) { |
||||
throw fatalException; |
||||
} |
||||
it.remove(); |
||||
throw e; |
||||
} catch (final KafkaException e) { |
||||
log.error("Failed to punctuate stream task {} due to the following error:", task.id(), e); |
||||
throw e; |
||||
} |
||||
} |
||||
return punctuated; |
||||
} |
||||
|
||||
} |
Loading…
Reference in new issue