-
Notifications
You must be signed in to change notification settings - Fork 4
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #116 from usdot-jpo-ode/dedup-spat
Dedup spat
- Loading branch information
Showing
10 changed files
with
336 additions
and
11 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
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
64 changes: 64 additions & 0 deletions
64
...main/java/us/dot/its/jpo/deduplicator/deduplicator/processors/ProcessedSpatProcessor.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,64 @@ | ||
package us.dot.its.jpo.deduplicator.deduplicator.processors; | ||
|
||
|
||
import java.time.Duration; | ||
import java.time.Instant; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
|
||
import us.dot.its.jpo.deduplicator.DeduplicatorProperties; | ||
import us.dot.its.jpo.geojsonconverter.pojos.spat.MovementEvent; | ||
import us.dot.its.jpo.geojsonconverter.pojos.spat.MovementState; | ||
import us.dot.its.jpo.geojsonconverter.pojos.spat.ProcessedSpat; | ||
|
||
public class ProcessedSpatProcessor extends DeduplicationProcessor<ProcessedSpat>{ | ||
|
||
DeduplicatorProperties props; | ||
|
||
public ProcessedSpatProcessor(DeduplicatorProperties props){ | ||
this.props = props; | ||
this.storeName = props.getKafkaStateStoreProcessedSpatName(); | ||
} | ||
|
||
|
||
@Override | ||
public Instant getMessageTime(ProcessedSpat message) { | ||
return message.getUtcTimeStamp().toInstant(); | ||
} | ||
|
||
@Override | ||
public boolean isDuplicate(ProcessedSpat lastMessage, ProcessedSpat newMessage) { | ||
|
||
Instant newValueTime = getMessageTime(newMessage); | ||
Instant oldValueTime = getMessageTime(lastMessage); | ||
|
||
if(newValueTime.minus(Duration.ofMinutes(1)).isAfter(oldValueTime)){ | ||
return false; | ||
}else{ | ||
HashMap<Integer, List<MovementEvent>> lastMessageStates = new HashMap<>(); | ||
for(MovementState state: lastMessage.getStates()){ | ||
lastMessageStates.put(state.getSignalGroup(), state.getStateTimeSpeed()); | ||
} | ||
|
||
if(lastMessageStates.size() != newMessage.getStates().size()){ | ||
return false; // message cannot be duplicate if the signal groups have a different number of signal groups | ||
} | ||
|
||
for(MovementState state: newMessage.getStates()){ | ||
List<MovementEvent> lastMessageState = lastMessageStates.get(state.getSignalGroup()); | ||
|
||
if(lastMessageState == null){ | ||
return false; // messages cannot be duplicates if they have different signal groups | ||
} | ||
|
||
|
||
for(int i=0; i< state.getStateTimeSpeed().size(); i++){ | ||
if(state.getStateTimeSpeed().get(i).getEventState() != lastMessageState.get(i).getEventState()){ | ||
return false; // Some signal group light has changed. Therefore the SPaTs are different | ||
} | ||
} | ||
} | ||
} | ||
return true; | ||
} | ||
} |
22 changes: 22 additions & 0 deletions
22
...ts/jpo/deduplicator/deduplicator/processors/suppliers/ProcessedSpatProcessorSupplier.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,22 @@ | ||
package us.dot.its.jpo.deduplicator.deduplicator.processors.suppliers; | ||
|
||
import org.apache.kafka.streams.processor.api.Processor; | ||
import org.apache.kafka.streams.processor.api.ProcessorSupplier; | ||
|
||
import us.dot.its.jpo.deduplicator.DeduplicatorProperties; | ||
import us.dot.its.jpo.deduplicator.deduplicator.processors.ProcessedSpatProcessor; | ||
import us.dot.its.jpo.geojsonconverter.pojos.spat.ProcessedSpat; | ||
|
||
public class ProcessedSpatProcessorSupplier implements ProcessorSupplier<String, ProcessedSpat, String, ProcessedSpat> { | ||
|
||
DeduplicatorProperties props; | ||
|
||
public ProcessedSpatProcessorSupplier(DeduplicatorProperties props){ | ||
this.props = props; | ||
} | ||
|
||
@Override | ||
public Processor<String, ProcessedSpat, String, ProcessedSpat> get() { | ||
return new ProcessedSpatProcessor(props); | ||
} | ||
} |
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
97 changes: 97 additions & 0 deletions
97
...s/dot/its/jpo/deduplicator/deduplicator/topologies/ProcessedSpatDeduplicatorTopology.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,97 @@ | ||
package us.dot.its.jpo.deduplicator.deduplicator.topologies; | ||
|
||
import org.apache.kafka.common.serialization.Serdes; | ||
import org.apache.kafka.streams.KafkaStreams; | ||
import org.apache.kafka.streams.StreamsBuilder; | ||
import org.apache.kafka.streams.Topology; | ||
import org.apache.kafka.streams.KafkaStreams.StateListener; | ||
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; | ||
|
||
import us.dot.its.jpo.deduplicator.DeduplicatorProperties; | ||
import us.dot.its.jpo.geojsonconverter.DateJsonMapper; | ||
import us.dot.its.jpo.deduplicator.deduplicator.processors.suppliers.ProcessedSpatProcessorSupplier; | ||
import us.dot.its.jpo.geojsonconverter.pojos.spat.ProcessedSpat; | ||
import us.dot.its.jpo.geojsonconverter.serialization.JsonSerdes; | ||
|
||
import org.apache.kafka.streams.kstream.*; | ||
import org.apache.kafka.streams.state.Stores; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import com.fasterxml.jackson.databind.ObjectMapper; | ||
|
||
import java.time.format.DateTimeFormatter; | ||
import java.util.Properties; | ||
|
||
public class ProcessedSpatDeduplicatorTopology { | ||
|
||
private static final Logger logger = LoggerFactory.getLogger(ProcessedSpatDeduplicatorTopology.class); | ||
|
||
Topology topology; | ||
KafkaStreams streams; | ||
String inputTopic; | ||
String outputTopic; | ||
Properties streamsProperties; | ||
ObjectMapper objectMapper; | ||
DateTimeFormatter formatter = DateTimeFormatter.ISO_INSTANT; | ||
DeduplicatorProperties props; | ||
|
||
|
||
public ProcessedSpatDeduplicatorTopology(DeduplicatorProperties props, Properties streamsProperties){ | ||
this.props = props; | ||
this.streamsProperties = streamsProperties; | ||
this.objectMapper = DateJsonMapper.getInstance(); | ||
} | ||
|
||
|
||
|
||
public void start() { | ||
if (streams != null && streams.state().isRunningOrRebalancing()) { | ||
throw new IllegalStateException("Start called while streams is already running."); | ||
} | ||
Topology topology = buildTopology(); | ||
streams = new KafkaStreams(topology, streamsProperties); | ||
if (exceptionHandler != null) streams.setUncaughtExceptionHandler(exceptionHandler); | ||
if (stateListener != null) streams.setStateListener(stateListener); | ||
logger.info("Starting Map Deduplicator Topology"); | ||
streams.start(); | ||
} | ||
|
||
public Topology buildTopology() { | ||
StreamsBuilder builder = new StreamsBuilder(); | ||
|
||
KStream<String, ProcessedSpat> inputStream = builder.stream(props.getKafkaTopicProcessedSpat(), Consumed.with(Serdes.String(), JsonSerdes.ProcessedSpat())); | ||
|
||
builder.addStateStore(Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore(props.getKafkaStateStoreProcessedSpatName()), | ||
Serdes.String(), JsonSerdes.ProcessedSpat())); | ||
|
||
KStream<String, ProcessedSpat> deduplicatedStream = inputStream.process(new ProcessedSpatProcessorSupplier(props), props.getKafkaStateStoreProcessedSpatName()); | ||
|
||
deduplicatedStream.print(Printed.toSysOut()); | ||
|
||
deduplicatedStream.to(props.getKafkaTopicDeduplicatedProcessedSpat(), Produced.with(Serdes.String(), JsonSerdes.ProcessedSpat())); | ||
|
||
return builder.build(); | ||
|
||
} | ||
|
||
public void stop() { | ||
logger.info("Stopping Processed SPaT deduplicator Socket Broadcast Topology."); | ||
if (streams != null) { | ||
streams.close(); | ||
streams.cleanUp(); | ||
streams = null; | ||
} | ||
logger.info("Stopped Processed SPaT deduplicator Socket Broadcast Topology."); | ||
} | ||
|
||
StateListener stateListener; | ||
public void registerStateListener(StateListener stateListener) { | ||
this.stateListener = stateListener; | ||
} | ||
|
||
StreamsUncaughtExceptionHandler exceptionHandler; | ||
public void registerUncaughtExceptionHandler(StreamsUncaughtExceptionHandler exceptionHandler) { | ||
this.exceptionHandler = exceptionHandler; | ||
} | ||
} |
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.