forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
guozhangwang * added a new config param "num.standby.replicas" (the default value is 0). * added a new abstract class AbstractTask * added StandbyTask as a subclass of AbstractTask * modified StreamTask to a subclass of AbstractTask * StreamThread * standby tasks are created by calling StreamThread.addStandbyTask() from onPartitionsAssigned() * standby tasks are destroyed by calling StreamThread.removeStandbyTasks() from onPartitionRevoked() * In addStandbyTasks(), change log partitions are assigned to restoreConsumer. * In removeStandByTasks(), change log partitions are removed from restoreConsumer. * StreamThread polls change log records using restoreConsumer in the runLoop with timeout=0. * If records are returned, StreamThread calls StandbyTask.update and pass records to each standby tasks. Author: Yasuhiro Matsuda <[email protected]> Reviewers: Guozhang Wang Closes apache#526 from ymatsuda/standby_task
- Loading branch information
1 parent
356544c
commit 45e7f71
Showing
12 changed files
with
951 additions
and
181 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
93 changes: 93 additions & 0 deletions
93
streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
/** | ||
* 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.clients.consumer.Consumer; | ||
import org.apache.kafka.common.KafkaException; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.streams.StreamingConfig; | ||
import org.apache.kafka.streams.processor.ProcessorContext; | ||
import org.apache.kafka.streams.processor.StateStore; | ||
import org.apache.kafka.streams.processor.StateStoreSupplier; | ||
import org.apache.kafka.streams.processor.TaskId; | ||
|
||
import java.io.File; | ||
import java.io.IOException; | ||
import java.util.Collections; | ||
import java.util.Set; | ||
|
||
public abstract class AbstractTask { | ||
protected final TaskId id; | ||
protected final ProcessorTopology topology; | ||
protected final ProcessorStateManager stateMgr; | ||
protected final Set<TopicPartition> partitions; | ||
protected ProcessorContext processorContext; | ||
|
||
protected AbstractTask(TaskId id, | ||
Consumer<byte[], byte[]> restoreConsumer, | ||
ProcessorTopology topology, | ||
StreamingConfig config, | ||
Set<TopicPartition> partitions) { | ||
this.id = id; | ||
this.topology = topology; | ||
this.partitions = partitions; | ||
|
||
// create the processor state manager | ||
try { | ||
File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), id.toString()); | ||
// if partitions is null, this is a standby task | ||
this.stateMgr = new ProcessorStateManager(id.partition, stateFile, restoreConsumer, partitions == null); | ||
} catch (IOException e) { | ||
throw new KafkaException("Error while creating the state manager", e); | ||
} | ||
} | ||
|
||
protected void initializeStateStores() { | ||
for (StateStoreSupplier stateStoreSupplier : this.topology.stateStoreSuppliers()) { | ||
StateStore store = stateStoreSupplier.get(); | ||
store.init(this.processorContext); | ||
} | ||
} | ||
|
||
public final TaskId id() { | ||
return id; | ||
} | ||
|
||
public final Set<TopicPartition> partitions() { | ||
return this.partitions; | ||
} | ||
|
||
public final ProcessorTopology topology() { | ||
return topology; | ||
} | ||
|
||
public final ProcessorContext context() { | ||
return processorContext; | ||
} | ||
|
||
public abstract void commit(); | ||
|
||
public void close() { | ||
try { | ||
stateMgr.close(Collections.<TopicPartition, Long>emptyMap()); | ||
} catch (IOException e) { | ||
throw new KafkaException("Error while closing the state manager in processor context", e); | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.