Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,8 @@ public interface StateStore {
*/
boolean isOpen();

void open(final StateStoreContext stateStoreContext);

/**
* Execute a query. Returns a QueryResult containing either result data or
* a failure.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,11 @@ public void close() {
throw new UnsupportedOperationException(ERROR_MESSAGE);
}

@Override
public void open(final StateStoreContext stateStoreContext) {
throw new UnsupportedOperationException(ERROR_MESSAGE);
}

static StateStore getReadOnlyStore(final StateStore global) {
if (global instanceof TimestampedKeyValueStore) {
return new TimestampedKeyValueStoreReadOnlyDecorator<>((TimestampedKeyValueStore<?, ?>) global);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,11 @@ public void close() {
throw new UnsupportedOperationException(ERROR_MESSAGE);
}

@Override
public void open(final StateStoreContext stateStoreContext) {
throw new UnsupportedOperationException(ERROR_MESSAGE);
}

static StateStore wrapWithReadWriteStore(final StateStore store) {
if (store instanceof TimestampedKeyValueStore) {
return new TimestampedKeyValueStoreReadWriteDecorator<>((TimestampedKeyValueStore<?, ?>) store);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@ public Set<String> initialize() {
for (final StateStore stateStore : topology.globalStateStores()) {
final String sourceTopic = storeToChangelogTopic.get(stateStore.name());
changelogTopics.add(sourceTopic);
stateStore.open(globalProcessorContext);
stateStore.init(globalProcessorContext, stateStore);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.ConsumerRecord;
import org.apache.kafka.common.header.internals.RecordHeaders;


public class InitProcessorRecordContext extends ProcessorRecordContext {

private final long initTime;
private static final long NO_OFFSET = -1;
private static final int NO_PARTITION = -1;

public InitProcessorRecordContext(final long currentTimestamp) {
super(ConsumerRecord.NO_TIMESTAMP, NO_OFFSET, NO_PARTITION, null, new RecordHeaders());
this.initTime = currentTimestamp;
}

@Override
public long timestamp() {
return initTime;
}

@Override
@Deprecated
public boolean equals(final Object o) {
return super.equals(o);
}

@Override
@Deprecated
public int hashCode() {
return super.hashCode();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,7 @@ void registerStateStores(final List<StateStore> allStores, final InternalProcess
maybeRegisterStoreWithChangelogReader(store.name());
}
} else {
store.init(processorContext, store);
store.open(processorContext);
}
log.trace("Registered state store {}", store.name());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ public Set<String> sourceTopics() {
return sourceNodesByTopic.get(topic);
}

public Map<String, SourceNode<?, ?>> getSourceNodesByTopic() {
return sourceNodesByTopic;
}

public Set<SourceNode<?, ?>> sources() {
return new HashSet<>(sourceNodesByTopic.values());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.errors.TaskCorruptedException;
import org.apache.kafka.streams.errors.TaskMigratedException;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
Expand Down Expand Up @@ -110,7 +111,9 @@ public void recordRestoration(final Time time, final long numRecords, final bool
public void initializeIfNeeded() {
if (state() == State.CREATED) {
StateManagerUtil.registerStateStores(log, logPrefix, topology, stateMgr, stateDirectory, processorContext);

for (final StateStore stateStore : topology.stateStores()) {
stateStore.init(processorContext, stateStore);
}
// with and without EOS we would check for checkpointing at each commit during running,
// and the file may be deleted in which case we should checkpoint immediately,
// therefore we initialize the snapshot as empty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -165,6 +165,34 @@ StandbyTask createStandbyTask(final TaskId taskId,
return task;
}

StandbyTask createStandbyTaskFromStartupLocalStore(final TaskId taskId,
final Set<TopicPartition> inputPartitions,
final ProcessorTopology topology,
final ProcessorStateManager stateManager) {
final InternalProcessorContext<Object, Object> context = new ProcessorContextImpl(
taskId,
applicationConfig,
stateManager,
streamsMetrics,
dummyCache
);
final StandbyTask task = new StandbyTask(
taskId,
inputPartitions,
topology,
topologyMetadata.taskConfig(taskId),
streamsMetrics,
stateManager,
stateDirectory,
dummyCache,
context
);

log.trace("Created standby task {} with assigned partitions {}", taskId, inputPartitions);
createTaskSensor.record();
return task;
}

private LogContext getLogContext(final TaskId taskId) {
final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final String logPrefix = threadIdPrefix + String.format("%s [%s] ", "standby-task", taskId);
Expand Down
Loading