@ -19,7 +19,6 @@ package org.apache.kafka.streams.kstream.internals;
import org.apache.kafka.common.serialization.Serde ;
import org.apache.kafka.common.serialization.Serde ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.errors.StreamsException ;
import org.apache.kafka.streams.errors.TopologyBuilderException ;
import org.apache.kafka.streams.errors.TopologyBuilderException ;
import org.apache.kafka.streams.kstream.ForeachAction ;
import org.apache.kafka.streams.kstream.ForeachAction ;
import org.apache.kafka.streams.kstream.KGroupedTable ;
import org.apache.kafka.streams.kstream.KGroupedTable ;
@ -31,14 +30,11 @@ import org.apache.kafka.streams.kstream.Predicate;
import org.apache.kafka.streams.kstream.ValueJoiner ;
import org.apache.kafka.streams.kstream.ValueJoiner ;
import org.apache.kafka.streams.kstream.ValueMapper ;
import org.apache.kafka.streams.kstream.ValueMapper ;
import org.apache.kafka.streams.processor.ProcessorSupplier ;
import org.apache.kafka.streams.processor.ProcessorSupplier ;
import org.apache.kafka.streams.processor.StateStoreSupplier ;
import org.apache.kafka.streams.processor.StreamPartitioner ;
import org.apache.kafka.streams.processor.StreamPartitioner ;
import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreSupplier ;
import java.io.FileNotFoundException ;
import java.io.FileNotFoundException ;
import java.io.FileOutputStream ;
import java.io.FileOutputStream ;
import java.io.PrintStream ;
import java.io.PrintStream ;
import java.util.Collections ;
import java.util.Objects ;
import java.util.Objects ;
import java.util.Set ;
import java.util.Set ;
@ -371,9 +367,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
KTableValueGetterSupplier < K , V > valueGetterSupplier ( ) {
KTableValueGetterSupplier < K , V > valueGetterSupplier ( ) {
if ( processorSupplier instanceof KTableSource ) {
if ( processorSupplier instanceof KTableSource ) {
KTableSource < K , V > source = ( KTableSource < K , V > ) processorSupplier ;
KTableSource < K , V > source = ( KTableSource < K , V > ) processorSupplier ;
if ( ! source . isMaterialized ( ) ) {
throw new StreamsException ( "Source is not materialized" ) ;
}
return new KTableSourceValueGetterSupplier < > ( source . storeName ) ;
return new KTableSourceValueGetterSupplier < > ( source . storeName ) ;
} else if ( processorSupplier instanceof KStreamAggProcessorSupplier ) {
} else if ( processorSupplier instanceof KStreamAggProcessorSupplier ) {
return ( ( KStreamAggProcessorSupplier < ? , K , S , V > ) processorSupplier ) . view ( ) ;
return ( ( KStreamAggProcessorSupplier < ? , K , S , V > ) processorSupplier ) . view ( ) ;
@ -387,9 +380,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
if ( ! sendOldValues ) {
if ( ! sendOldValues ) {
if ( processorSupplier instanceof KTableSource ) {
if ( processorSupplier instanceof KTableSource ) {
KTableSource < K , ? > source = ( KTableSource < K , V > ) processorSupplier ;
KTableSource < K , ? > source = ( KTableSource < K , V > ) processorSupplier ;
if ( ! source . isMaterialized ( ) ) {
throw new StreamsException ( "Source is not materialized" ) ;
}
source . enableSendingOldValues ( ) ;
source . enableSendingOldValues ( ) ;
} else if ( processorSupplier instanceof KStreamAggProcessorSupplier ) {
} else if ( processorSupplier instanceof KStreamAggProcessorSupplier ) {
( ( KStreamAggProcessorSupplier < ? , K , S , V > ) processorSupplier ) . enableSendingOldValues ( ) ;
( ( KStreamAggProcessorSupplier < ? , K , S , V > ) processorSupplier ) . enableSendingOldValues ( ) ;
@ -404,19 +394,4 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
return sendOldValues ;
return sendOldValues ;
}
}
public void materialize ( KTableSource < K , ? > source ) {
synchronized ( source ) {
if ( ! source . isMaterialized ( ) ) {
StateStoreSupplier storeSupplier = new RocksDBKeyValueStoreSupplier < > ( source . storeName ,
keySerde ,
valSerde ,
false ,
Collections . < String , String > emptyMap ( ) ,
true ) ;
// mark this state as non internal hence it is read directly from a user topic
topology . addStateStore ( storeSupplier , name ) ;
source . materialize ( ) ;
}
}
}
}
}