Skip to content

Commit

Permalink
Fix: Remove expensive changelog topics (#37)
Browse files Browse the repository at this point in the history
* feat: configure stores for lower starting times and less duplicated data

* feat: configure stores for lower starting times and less duplicated data

* chore: enabled easier tagging by vars

* feat: configure stores for lower starting times and less duplicated data

* fix: start stream before sending messages

* chore: use always master, pass other as args
  • Loading branch information
jeqo committed Sep 3, 2019
1 parent f1a804c commit bbdf04c
Show file tree
Hide file tree
Showing 5 changed files with 115 additions and 147 deletions.
2 changes: 1 addition & 1 deletion Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,8 @@

FROM alpine

ARG VERSION=master-SNAPSHOT
ENV USER jeqo
ENV VERSION 0.5.3

WORKDIR /zipkin

Expand Down
8 changes: 4 additions & 4 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ all: build

OPEN := 'xdg-open'
MAVEN := './mvnw'
VERSION := '0.5.4-SNAPSHOT'
VERSION := 'master-SNAPSHOT'
IMAGE_NAME := 'jeqo/zipkin-kafka'

.PHONY: run
Expand All @@ -23,11 +23,11 @@ kafka-topics:

.PHONY: docker-build
docker-build:
docker build -t ${IMAGE_NAME}:latest .
docker build -t ${IMAGE_NAME}:${VERSION} .
docker build --build-arg VERSION=${VERSION} -t ${IMAGE_NAME}:latest .
docker build --build-arg VERSION=${VERSION} -t ${IMAGE_NAME}:${VERSION} .

.PHONY: docker-push
docker-push: docker-build
docker-push:
docker push ${IMAGE_NAME}:latest
docker push ${IMAGE_NAME}:${VERSION}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,15 +55,17 @@ public DependencyStoreTopologySupplier(String dependencyTopicName,
StreamsBuilder builder = new StreamsBuilder();

// Dependency links window store
builder.addStateStore(Stores.windowStoreBuilder(
Stores.persistentWindowStore(
DEPENDENCIES_STORE_NAME,
dependencyTtl,
dependencyWindowSize,
false),
Serdes.String(),
dependencyLinkSerde
));
builder.addStateStore(
// Disabling logging to avoid long starting times
Stores.windowStoreBuilder(
Stores.persistentWindowStore(
DEPENDENCIES_STORE_NAME,
dependencyTtl,
dependencyWindowSize,
false),
Serdes.String(),
dependencyLinkSerde
).withLoggingDisabled());
// Consume dependency links stream
builder.stream(dependencyTopicName, Consumed.with(Serdes.String(), dependencyLinkSerde))
// Storage
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.util.Set;
import java.util.function.Supplier;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.Consumed;
Expand Down Expand Up @@ -79,99 +78,104 @@ public TraceStoreTopologySupplier(String spansTopicName, List<String> autoComple
@Override public Topology get() {
StreamsBuilder builder = new StreamsBuilder();
builder
// Logging disabled to avoid long starting times
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(TRACES_STORE_NAME),
Serdes.String(),
spansSerde))
spansSerde).withLoggingDisabled())
// Disabling logging to avoid long starting times
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(SPAN_IDS_BY_TS_STORE_NAME),
Serdes.Long(),
spanIdsSerde))
spanIdsSerde).withLoggingDisabled())
// In-memory as service names are bounded
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(SERVICE_NAMES_STORE_NAME),
Stores.inMemoryKeyValueStore(SERVICE_NAMES_STORE_NAME),
Serdes.String(),
Serdes.String()))
// In-memory as span names are bounded
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(SPAN_NAMES_STORE_NAME),
Stores.inMemoryKeyValueStore(SPAN_NAMES_STORE_NAME),
Serdes.String(),
namesSerde))
// In-memory as remote-service names are bounded
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(REMOTE_SERVICE_NAMES_STORE_NAME),
Stores.inMemoryKeyValueStore(REMOTE_SERVICE_NAMES_STORE_NAME),
Serdes.String(),
namesSerde))
// Persistent as values could be unbounded
.addStateStore(Stores.keyValueStoreBuilder(
Stores.persistentKeyValueStore(AUTOCOMPLETE_TAGS_STORE_NAME),
Serdes.String(),
namesSerde));
namesSerde).withLoggingDisabled());
// Traces stream
KStream<String, List<Span>> spansStream = builder
.stream(spansTopicName, Consumed.with(Serdes.String(), spansSerde));
.stream(
spansTopicName,
Consumed.with(Serdes.String(), spansSerde)
.withOffsetResetPolicy(Topology.AutoOffsetReset.LATEST));
// Store traces
spansStream.process(() -> new Processor<String, List<Span>>() {
ProcessorContext context;
// Actual traces store
KeyValueStore<String, List<Span>> tracesStore;
// timestamp index for trace IDs
KeyValueStore<Long, Set<String>> spanIdsByTsStore;
ProcessorContext context;
// Actual traces store
KeyValueStore<String, List<Span>> tracesStore;
// timestamp index for trace IDs
KeyValueStore<Long, Set<String>> spanIdsByTsStore;

@Override public void init(ProcessorContext context) {
this.context = context;
tracesStore =
(KeyValueStore<String, List<Span>>) context.getStateStore(TRACES_STORE_NAME);
spanIdsByTsStore =
(KeyValueStore<Long, Set<String>>) context.getStateStore(SPAN_IDS_BY_TS_STORE_NAME);
// Retention scheduling
context.schedule(
traceTtlCheckInterval,
PunctuationType.STREAM_TIME,
timestamp -> {
if (traceTtl.toMillis() > 0 &&
tracesStore.approximateNumEntries() > minTracesStored) {
// preparing range filtering
long from = 0L;
long to = timestamp - traceTtl.toMillis();
long toMicro = to * 1000;
// query traceIds active during period
try (final KeyValueIterator<Long, Set<String>> all =
spanIdsByTsStore.range(from, toMicro)) {
int deletions = 0; // logging purpose
while (all.hasNext()) {
final KeyValue<Long, Set<String>> record = all.next();
spanIdsByTsStore.delete(record.key); // clean timestamp index
for (String traceId : record.value) {
tracesStore.delete(traceId); // clean traces store
deletions++;
}
}
if (deletions > 0) {
LOG.info("Traces deletion emitted: {}, older than {}",
deletions,
Instant.ofEpochMilli(to).atZone(ZoneId.systemDefault()));
}
@Override public void init(ProcessorContext context) {
this.context = context;
tracesStore =
(KeyValueStore<String, List<Span>>) context.getStateStore(TRACES_STORE_NAME);
spanIdsByTsStore =
(KeyValueStore<Long, Set<String>>) context.getStateStore(SPAN_IDS_BY_TS_STORE_NAME);
// Retention scheduling
context.schedule(
traceTtlCheckInterval,
PunctuationType.STREAM_TIME,
timestamp -> {
if (traceTtl.toMillis() > 0 &&
tracesStore.approximateNumEntries() > minTracesStored) {
// preparing range filtering
long from = 0L;
long to = timestamp - traceTtl.toMillis();
long toMicro = to * 1000;
// query traceIds active during period
try (final KeyValueIterator<Long, Set<String>> range =
spanIdsByTsStore.range(from, toMicro)) {
range.forEachRemaining(record -> {
spanIdsByTsStore.delete(record.key); // clean timestamp index
for (String traceId : record.value) {
tracesStore.delete(traceId); // clean traces store
}
}
});
}
});
LOG.info("Traces deletion emitted at {}, approx. number of traces stored {} - partition: {}",
Instant.ofEpochMilli(to).atZone(ZoneId.systemDefault()),
tracesStore.approximateNumEntries(),
context.partition());
}
}
});
}

@Override public void process(String traceId, List<Span> spans) {
if (!spans.isEmpty()) {
// Persist traces
List<Span> currentSpans = tracesStore.get(traceId);
if (currentSpans == null) currentSpans = new ArrayList<>();
currentSpans.addAll(spans);
tracesStore.put(traceId, currentSpans);
// Persist timestamp indexed span ids
long timestamp = spans.get(0).timestamp();
Set<String> currentSpanIds = spanIdsByTsStore.get(timestamp);
if (currentSpanIds == null) currentSpanIds = new HashSet<>();
currentSpanIds.add(traceId);
spanIdsByTsStore.put(timestamp, currentSpanIds);
}
}
@Override public void process(String traceId, List<Span> spans) {
if (!spans.isEmpty()) {
// Persist traces
List<Span> currentSpans = tracesStore.get(traceId);
if (currentSpans == null) currentSpans = new ArrayList<>();
currentSpans.addAll(spans);
tracesStore.put(traceId, currentSpans);
// Persist timestamp indexed span ids
long timestamp = spans.get(0).timestamp();
Set<String> currentSpanIds = spanIdsByTsStore.get(timestamp);
if (currentSpanIds == null) currentSpanIds = new HashSet<>();
currentSpanIds.add(traceId);
spanIdsByTsStore.put(timestamp, currentSpanIds);
}
}

@Override public void close() {
}
}, TRACES_STORE_NAME, SPAN_IDS_BY_TS_STORE_NAME);
@Override public void close() {
}
}, TRACES_STORE_NAME, SPAN_IDS_BY_TS_STORE_NAME);
// Store service, span and remote service names
spansStream.process(() -> new Processor<String, List<Span>>() {
KeyValueStore<String, String> serviceNameStore;
Expand Down
92 changes: 27 additions & 65 deletions storage/src/test/java/zipkin2/storage/kafka/KafkaStorageIT.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.errors.InvalidStateStoreException;
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -163,6 +164,11 @@ class KafkaStorageIT {
.timestamp(TODAY * 1000 + 10).duration(8)
.build();
List<Span> spans = Arrays.asList(parent, child);
// When: and stores running
SpanStore spanStore = storage.spanStore();
ServiceAndSpanNames serviceAndSpanNames = storage.serviceAndSpanNames();
await().atMost(10, TimeUnit.SECONDS)
.until(() -> storage.traceStoreStream.state().equals(KafkaStreams.State.RUNNING));
// When: been published
tracesProducer.send(new ProducerRecord<>(storage.spansTopicName, parent.traceId(), spans));
tracesProducer.send(new ProducerRecord<>(storage.spansTopicName, other.traceId(),
Expand All @@ -171,76 +177,32 @@ class KafkaStorageIT {
// Then: stored
IntegrationTestUtils.waitUntilMinRecordsReceived(
testConsumerConfig, storage.spansTopicName, 1, 10000);
// When: and stores running
SpanStore spanStore = storage.spanStore();
ServiceAndSpanNames serviceAndSpanNames = storage.serviceAndSpanNames();
// Then: services names are searchable
await().atMost(10, TimeUnit.SECONDS)
.until(() -> {
List<List<Span>> traces = new ArrayList<>();
try {
traces =
spanStore.getTraces(QueryRequest.newBuilder()
.endTs(TODAY + 1)
.lookback(Duration.ofSeconds(30).toMillis())
.serviceName("svc_a")
.limit(10)
.build())
.execute();
} catch (InvalidStateStoreException e) { // ignoring state issues
System.err.println(e.getMessage());
} catch (Exception e) {
e.printStackTrace();
}
return traces.size() == 1
&& traces.get(0).size() == 2; // Trace is found and has two spans
}); // wait for services to be available
List<List<Span>> traces = new ArrayList<>();
try {
traces =
spanStore.getTraces(QueryRequest.newBuilder()
.endTs(TODAY + 1)
.lookback(Duration.ofMinutes(1).toMillis())
.limit(1)
.build())
.execute();
} catch (InvalidStateStoreException e) { // ignoring state issues
System.err.println(e.getMessage());
} catch (Exception e) {
e.printStackTrace();
}
List<List<Span>> traces = spanStore.getTraces(QueryRequest.newBuilder()
.endTs(TODAY + 1)
.lookback(Duration.ofSeconds(30).toMillis())
.serviceName("svc_a")
.limit(10)
.build())
.execute();
assertEquals(1, traces.size());
assertEquals(1, traces.get(0).size()); // last trace is returned first

List<String> services = new ArrayList<>();
try {
services = serviceAndSpanNames.getServiceNames().execute();
} catch (InvalidStateStoreException e) { // ignoring state issues
System.err.println(e.getMessage());
} catch (Exception e) {
e.printStackTrace();
}
assertEquals(traces.get(0).size(), 2); // Trace is found and has two spans
List<List<Span>> filteredTraces =
spanStore.getTraces(QueryRequest.newBuilder()
.endTs(TODAY + 1)
.lookback(Duration.ofMinutes(1).toMillis())
.limit(1)
.build())
.execute();
assertEquals(1, filteredTraces.size());
assertEquals(1, filteredTraces.get(0).size()); // last trace is returned first
List<String> services = serviceAndSpanNames.getServiceNames().execute();
assertEquals(3, services.size()); // There are two service names

List<String> spanNames = new ArrayList<>();
try {
spanNames = serviceAndSpanNames.getSpanNames("svc_a")
.execute();
} catch (InvalidStateStoreException e) { // ignoring state issues
System.err.println(e.getMessage());
} catch (Exception e) {
e.printStackTrace();
}
List<String> spanNames = serviceAndSpanNames.getSpanNames("svc_a")
.execute();
assertEquals(1, spanNames.size());// Service names have one span name

List<String> remoteServices = new ArrayList<>();
try {
remoteServices = serviceAndSpanNames.getRemoteServiceNames("svc_a").execute();
} catch (InvalidStateStoreException e) { // ignoring state issues
System.err.println(e.getMessage());
} catch (Exception e) {
e.printStackTrace();
}
List<String> remoteServices = serviceAndSpanNames.getRemoteServiceNames("svc_a").execute();
assertEquals(1, remoteServices.size());// And one remote service name
}

Expand Down

0 comments on commit bbdf04c

Please sign in to comment.