From 85ea47530949585f48c847137b4095ff6f5292f4 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 13 Mar 2019 23:06:54 -0700 Subject: [PATCH 01/78] Issue 191: Add non-Flink stream processing example. Signed-off-by: Claudio Fahey --- README.md | 1 + pravega-client-examples/README.md | 3 + pravega-client-examples/build.gradle | 14 + .../streamprocessing/EventDebugSink.java | 95 ++++ .../streamprocessing/EventGenerator.java | 98 ++++ .../ExactlyOnceMultithreadedProcessor.java | 495 ++++++++++++++++++ .../NonRecoverableMultithreadedProcessor.java | 209 ++++++++ ...NonRecoverableSingleThreadedProcessor.java | 163 ++++++ .../example/streamprocessing/Parameters.java | 66 +++ .../example/streamprocessing/README.md | 177 +++++++ .../RecoverableMultithreadedProcessor.java | 279 ++++++++++ 11 files changed, 1600 insertions(+) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java diff --git a/README.md b/README.md index 5cf200a2..dec7f028 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ more complex applications as sub-projects, which show use-cases exploiting one o | `noop` | Example of how to add a simple callback executed upon a read event. | [Java](pravega-client-examples/src/main/java/io/pravega/example/noop) | `statesynchronizer` | Application that allows users to work with `StateSynchronizer` API via CLI. | [Java](pravega-client-examples/src/main/java/io/pravega/example/statesynchronizer) | `streamcuts` | Application examples demonstrating the use of `StreamCut`s via CLI. | [Java](pravega-client-examples/src/main/java/io/pravega/example/streamcuts) +| `streamprocessing` | An example that illustrates exactly-once processing using the Pravega API. | [Java](pravega-client-examples/src/main/java/io/pravega/example/streamprocessing) The related documentation and instructions are [here](pravega-client-examples). diff --git a/pravega-client-examples/README.md b/pravega-client-examples/README.md index 3a3cdb30..e73ee7e8 100644 --- a/pravega-client-examples/README.md +++ b/pravega-client-examples/README.md @@ -198,3 +198,6 @@ $ bin/secureBatchReader [-scope "myScope"] [-stream "myStream"] [-uri "tls://loc All args are optional. If not included, the default values are same as the defaults mentioned earlier for `bin\secureWriter`. + +## `streamprocessing` +See [streamprocessing](src/main/java/io/pravega/example/streamprocessing/README.md). diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index c903ee47..d73cb1da 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -182,6 +182,20 @@ task startSecureBatchReader(type: JavaExec) { } } +task startEventGenerator(type: JavaExec) { + main = "io.pravega.example.streamprocessing.EventGenerator" + classpath = sourceSets.main.runtimeClasspath +} + +task startExactlyOnceMultithreadedProcessor(type: JavaExec) { + main = "io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor" + classpath = sourceSets.main.runtimeClasspath +} + +task startEventDebugSink(type: JavaExec) { + main = "io.pravega.example.streamprocessing.EventDebugSink" + classpath = sourceSets.main.runtimeClasspath +} distributions { main { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java new file mode 100644 index 00000000..174d38d1 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -0,0 +1,95 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.util.UUID; + +/** + * A simple example that continuously shows the events in a stream. + * + * This reads the output of {@link ExactlyOnceMultithreadedProcessor}. + */ +public class EventDebugSink { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventDebugSink.class); + + private static final int READER_TIMEOUT_MS = 2000; + + public final String scope; + public final String inputStreamName; + public final URI controllerURI; + + public EventDebugSink(String scope, String inputStreamName, URI controllerURI) { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.controllerURI = controllerURI; + } + + public static void main(String[] args) throws Exception { + EventDebugSink processor = new EventDebugSink( + Parameters.getScope(), + Parameters.getStream2Name(), + Parameters.getControllerURI()); + processor.run(); + } + + public void run() throws Exception { + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + } + + // Create a reader group that begins at the earliest event. + final String readerGroup = UUID.randomUUID().toString().replace("-", ""); + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, inputStreamName)) + .build(); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + } + + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new UTF8StringSerializer(), + ReaderConfig.builder().build())) { + long eventCounter = 0; + long sum = 0; + for (;;) { + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + if (eventRead.getEvent() != null) { + eventCounter++; + String[] cols = eventRead.getEvent().split(","); + long intData = Long.parseLong(cols[3]); + sum += intData; + log.info("eventCounter={}, sum={}, event={}", + String.format("%06d", eventCounter), + String.format("%08d", sum), + eventRead.getEvent()); + } + } + } + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java new file mode 100644 index 00000000..074c90d8 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -0,0 +1,98 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Random; +import java.util.concurrent.CompletableFuture; + +/** + * A simple example app that to write messages to a Pravega stream. + * + * Use {@link ExactlyOnceMultithreadedProcessor} to read output events. + */ +public class EventGenerator { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventGenerator.class); + + public final String scope; + public final String outputStreamName; + public final URI controllerURI; + + public EventGenerator(String scope, String outputStreamName, URI controllerURI) { + this.scope = scope; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + } + + public static void main(String[] args) throws Exception { + EventGenerator processor = new EventGenerator( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getControllerURI()); + processor.run(); + } + + public void run() throws Exception { + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + Random rand = new Random(42); + + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { + long eventCounter = 0; + long sum = 0; + for (;;) { + eventCounter++; + String routingKey = String.format("rk%02d", eventCounter % 10); + long intData = rand.nextInt(100); + sum += intData; + String generatedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + String message = String.join(",", + String.format("%06d", eventCounter), + routingKey, + String.format("%02d", intData), + String.format("%08d", sum), + generatedTimestampStr + ); + log.info("eventCounter={}, sum={}, event={}", + String.format("%06d", eventCounter), + String.format("%08d", sum), + message); + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + Thread.sleep(1000); + } + } + } + +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java new file mode 100644 index 00000000..805d5d88 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -0,0 +1,495 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.*; +import java.nio.file.attribute.BasicFileAttributes; +import java.text.SimpleDateFormat; +import java.util.*; +import java.util.concurrent.*; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * and writing each output event to another Pravega stream. + * + * This supports multiple worker threads. + * Upon restart, it restarts from the last successful checkpoint and guarantees exactly-once semantics. + * + * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} + * to view the output events. + */ +public class ExactlyOnceMultithreadedProcessor { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessor.class); + + private static final String CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX = "pravega-transactions-worker-"; + + public static class Master implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); + + private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; + private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; + + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; + private final int numWorkers; + private final String readerGroupName; + private final ReaderGroup readerGroup; + private final ScheduledExecutorService initiateCheckpointExecutor; + private final ScheduledExecutorService performCheckpointExecutor; + private final ExecutorService workerExecutor; + private final ReaderGroupManager readerGroupManager; + private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); + private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); + private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); + private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); + private final boolean startFromCheckpoint; + private final String startFromCheckpointName; + private final ClientFactory clientFactory; + private final EventStreamWriter writer; + + public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + this.numWorkers = numWorkers; + + ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() + .disableAutomaticCheckpoints(); + + // Load the last checkpoint. + startFromCheckpoint = latestCheckpointNamePath.toFile().exists(); + if (startFromCheckpoint) { + // Read the name of the checkpoint from the file /tmp/checkpoint/latest. + String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Reading Pravega checkpoint from {}", checkpointPath); + Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); + log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + builder = builder.startFromCheckpoint(checkpoint); + startFromCheckpointName = checkpointName; + } else { + log.warn("Checkpoint file {} not found. Starting processing from the earliest event.", checkpointRootPath); + + // Create streams. + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + // Since we are starting processing from the beginning, delete and create a new output stream. + // TODO: Should we truncate stream instead of deleting? + try { + streamManager.sealStream(scope, outputStreamName); + } catch (Exception e) { + if (!(e.getCause() instanceof InvalidStreamException)) { + throw e; + } + } + // TODO: It would be nice if deleteStream did not require sealStream to be called. + streamManager.deleteStream(scope, outputStreamName); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + // Create a reader group that starts from the earliest event. + builder = builder.stream(Stream.of(scope, inputStreamName)); + startFromCheckpointName = null; + } + + clientFactory = ClientFactory.withScope(scope, controllerURI); + writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build()); + + // Create a reader group manager. It must remain open to allow manual checkpoints to work. + readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); + + final ReaderGroupConfig readerGroupConfig = builder.build(); + readerGroupName = UUID.randomUUID().toString().replace("-", ""); + readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); + readerGroup = readerGroupManager.getReaderGroup(readerGroupName); + + initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); + performCheckpointExecutor = Executors.newScheduledThreadPool(1); + workerExecutor = Executors.newFixedThreadPool(numWorkers); + } + + /** + * Commit all transactions that are part of a checkpoint. + * + * @param checkpointName + */ + private void commitTransactions(String checkpointName) { + log.info("commitTransactions: BEGIN"); + + // Read the contents of all pravega-transactions-worker-XX files. + // These files contain the Pravega transaction IDs that must be committed now. + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + List txnIds = IntStream + .range(0, numWorkers) + .boxed() + .map(workerIndex -> checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + workerIndex)) + .flatMap(path -> { + try { + return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .map(UUID::fromString) + .collect(Collectors.toList()); + + log.info("commitTransactions: txnIds={}", txnIds); + + // Initiate commit of all transactions in the checkpoint. + txnIds.parallelStream().forEach(txnId -> { + try { + Transaction transaction = writer.getTxn(txnId); + Transaction.Status status = transaction.checkStatus(); + log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); + if (status == Transaction.Status.OPEN) { + log.info("commitTransaction: committing {}", transaction.getTxnId()); + transaction.commit(); + // Note that commit may return before the transaction is committed. + // TODO: It would be nice for commit() to return a future when it becomes COMMITTED or ABORTED. + } + } catch (TxnFailedException e) { + throw new RuntimeException(e); + } + }); + + // Wait for commit of all transactions in the checkpoint. + txnIds.parallelStream().forEach(txnId -> { + try { + Transaction transaction = writer.getTxn(txnId); + // TODO: Is there a better way to wait for COMMITTED besides polling? + for (; ; ) { + Transaction.Status status = transaction.checkStatus(); + log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); + if (status == Transaction.Status.COMMITTED) { + log.info("commitTransaction: committed {}", transaction.getTxnId()); + break; + } else if (status == Transaction.Status.ABORTED) { + throw new RuntimeException(new TxnFailedException()); + } + Thread.sleep(100); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + log.info("commitTransactions: END"); + } + + public void run() { + // It is possible that the checkpoint was completely written but that some or all Pravega transactions + // have not been committed. This will ensure that they are. + if (startFromCheckpoint) { + commitTransactions(startFromCheckpointName); + } + + // Schedule periodic task to initiate checkpoints. + // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. + initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); + + // Start workers. + IntStream.range(0, numWorkers).forEach(workerIndex -> { + Worker worker = new Worker(workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI); + workerExecutor.submit(worker); + }); + try { + workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { + } + } + + /** + * Initiate a checkpoint, wait for it to complete, and write the checkpoint to the state. + */ + private void performCheckpoint() { + final String checkpointName = UUID.randomUUID().toString(); + log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); + try { + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + checkpointDirPath.toFile().mkdirs(); + + final Set onlineReaders = readerGroup.getOnlineReaders(); + log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); + log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); + CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); + Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); + // At this point, all workers have received and processed the checkpoint. + log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Writing Pravega checkpoint to {}", checkpointPath); + try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { + fos.write(checkpoint.toBytes().array()); + fos.flush(); + fos.getFD().sync(); + } + + // Create "latest" file that indicates the latest checkpoint name. + // This file must be updated atomically. + Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + ".tmp"); + try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { + fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); + fos.flush(); + fos.getFD().sync(); + } + Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); + + // Read list of TxnIds from checkpoint directory written by all workers and commit all transactions. + commitTransactions(checkpointName); + + cleanCheckpointDirectory(checkpointDirPath); + } catch (final Exception e) { + log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); + // Ignore error. We will retry when we are scheduled again. + } + log.info("performCheckpoint: END: checkpointName={}", checkpointName); + } + + /** + * Delete everything in the checkpoint root path (/tmp/checkpoint) except the "latest" file and + * the latest checkpoint directory. + * + * @param keepCheckpointDirPath The latest checkpoint directory which will not be deleted. + */ + private void cleanCheckpointDirectory(Path keepCheckpointDirPath) { + try { + Files.walkFileTree(checkpointRootPath, + new SimpleFileVisitor() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes basicFileAttributes) throws IOException { + if (dir.equals(keepCheckpointDirPath)) { + return FileVisitResult.SKIP_SUBTREE; + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + if (!dir.equals(checkpointRootPath)) { + Files.delete(dir); + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + if (!file.equals(latestCheckpointNamePath)) { + Files.delete(file); + } + return FileVisitResult.CONTINUE; + } + }); + } catch (IOException e) { + log.warn("cleanCheckpointDirectory", e); + } + } + } + + public static class Worker implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); + + private static final String STATE_FILE_NAME_PREFIX = "state-worker-"; + private static final int READER_TIMEOUT_MS = 2000; + + private static class State implements Serializable { + private static final long serialVersionUID = -275148988691911596L; + + long sum; + + public State() { + this.sum = 0; + } + + @Override + public String toString() { + return "State{" + + "sum=" + sum + + '}'; + } + } + + private final int workerIndex; + private final String scope; + private final String readerGroupName; + private final boolean startFromCheckpoint; + private final String startFromCheckpointName; + private final String outputStreamName; + private final URI controllerURI; + private final String readerId; + private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); + + private State state; + + public Worker(int workerIndex, String scope, String readerGroupName, String startFromCheckpointName, String outputStreamName, URI controllerURI) { + this.workerIndex = workerIndex; + this.scope = scope; + this.readerGroupName = readerGroupName; + this.startFromCheckpointName = startFromCheckpointName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + readerId = "worker-" + this.workerIndex; + startFromCheckpoint = startFromCheckpointName != null; + } + + public void run() { + Thread.currentThread().setName("worker-" + workerIndex); + log.info("BEGIN"); + + try { + // Load state from checkpoint. + if (startFromCheckpoint) { + Path checkpointDirPath = checkpointRootPath.resolve(startFromCheckpointName); + Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); + log.info("statePath={}", statePath.toString()); + try (FileInputStream fis = new FileInputStream(statePath.toString()); + ObjectInputStream ois = new ObjectInputStream(fis)) { + state = (State) ois.readObject(); + } + log.info("Loaded state {} from {}", state, statePath); + } else { + log.info("Initializing with new state"); + state = new State(); + } + + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + readerId, + readerGroupName, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder() + .transactionTimeoutTime(Parameters.getTransactionTimeoutMs()) + .build())) { + + Transaction transaction = null; + long eventCounter = 0; + + for (; ; ) { + // Read input event. + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + log.debug("readEvents: eventRead={}", eventRead); + + if (eventRead.isCheckpoint()) { + // Note that next call readNextEvent will indicate to Pravega that we are done with the checkpoint. + String checkpointName = eventRead.getCheckpointName(); + log.info("Got checkpoint {}", eventRead.getCheckpointName()); + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + Path transactionIdFilePath = checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + this.workerIndex); + + // Must ensure that txnId is persisted to latest state before committing transaction! + // Do not commit transaction here. Instead write TxnId to checkpoint directory. Master will read all TxnIds and commit transactions. + + String transactionIds = ""; + if (transaction != null) { + transaction.flush(); + transactionIds = transaction.getTxnId().toString(); + transaction = null; + } + Files.write(transactionIdFilePath, transactionIds.getBytes(StandardCharsets.UTF_8)); + + // Write state to checkpoint directory + Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); + log.info("statePath={}", statePath.toString()); + try (FileOutputStream fos = new FileOutputStream(statePath.toString()); + ObjectOutputStream oos = new ObjectOutputStream(fos)) { + oos.writeObject(state); + oos.flush(); + fos.getFD().sync(); + } + + } else if (eventRead.getEvent() != null) { + eventCounter++; + log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); + + if (transaction == null) { + transaction = writer.beginTxn(); + } + + // Parse input event. + String[] cols = eventRead.getEvent().split(","); + long generatedEventCounter = Long.parseLong(cols[0]); + String routingKey = cols[1]; + long intData = Long.parseLong(cols[2]); + long generatedSum = Long.parseLong(cols[3]); + String generatedTimestampStr = cols[4]; + + // Process the input event and update the state. + state.sum += intData; + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + String.format("%06d", generatedEventCounter), + String.format("%06d", eventCounter), + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedSum), + String.format("%08d", state.sum), + String.format("%03d", workerIndex), + generatedTimestampStr, + processedTimestampStr, + transaction.getTxnId().toString()); + + // Write the output event. + log.info("eventCounter={}, event={}", + String.format("%06d", eventCounter), + message); + transaction.writeEvent(routingKey, message); + } + } + } + } catch (Exception e) { + log.error("Fatal Error", e); + System.exit(1); + } + } + } + + public static void main(String[] args) throws Exception { + Master master = new Master( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getStream2Name(), + Parameters.getControllerURI(), + Parameters.getNumWorkers()); + master.run(); + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java new file mode 100644 index 00000000..03763f3a --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java @@ -0,0 +1,209 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.UUID; +import java.util.concurrent.*; +import java.util.stream.IntStream; + +/** + * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * and writing each output event to another Pravega stream. + * + * This supports multiple worker threads. + * Upon restart, it reprocesses the entire input stream and recreates the output stream. + * + * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} + * to view the output events. + * + * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. + */ +public class NonRecoverableMultithreadedProcessor { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(NonRecoverableMultithreadedProcessor.class); + + public static class Master implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); + + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; + private final int numWorkers; + private final String readerGroupName; + private final ReaderGroup readerGroup; + private final ExecutorService workerExecutor; + + public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + this.numWorkers = numWorkers; + + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + // Since we start reading the input stream from the earliest event, we must delete the output stream. + try { + streamManager.sealStream(scope, outputStreamName); + } catch (Exception e) { + if (!(e.getCause() instanceof InvalidStreamException)) { + throw e; + } + } + streamManager.deleteStream(scope, outputStreamName); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + // Create a reader group that begins at the earliest event. + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, inputStreamName)) + .build(); + readerGroupName = UUID.randomUUID().toString().replace("-", ""); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { + readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); + readerGroup = readerGroupManager.getReaderGroup(readerGroupName); + } + + workerExecutor = Executors.newFixedThreadPool(numWorkers); + } + + public void run() { + IntStream.range(0, numWorkers).forEach(workerIndex -> { + Worker worker = new Worker(workerIndex, scope, readerGroupName, outputStreamName, controllerURI); + workerExecutor.submit(worker); + }); + try { + workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { + } + } + } + + public static class Worker implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); + + private static class State { + long sum; + + public State(long sum) { + this.sum = sum; + } + } + + private static final int readerTimeoutMs = 2000; + + private final int workerIndex; + private final String scope; + private final String readerGroupName; + private final String outputStreamName; + private final URI controllerURI; + private final String readerId; + + private State state; + + public Worker(int workerIndex, String scope, String readerGroupName, String outputStreamName, URI controllerURI) { + this.workerIndex = workerIndex; + this.scope = scope; + this.readerGroupName = readerGroupName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + readerId = "reader" + this.workerIndex; + } + + public void run() { + Thread.currentThread().setName("worker-" + workerIndex); + log.info("BEGIN"); + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + readerId, + readerGroupName, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { + + // Initialize state. + state = new State(0); + + EventRead event; + for (int i = 0; ; i++) { + // Read input event. + try { + event = reader.readNextEvent(readerTimeoutMs); + } catch (ReinitializationRequiredException e) { + // There are certain circumstances where the reader needs to be reinitialized + log.error("Read error", e); + throw new RuntimeException(e); + } + + if (event.getEvent() != null) { + log.info("Read event '{}'", event.getEvent()); + + // Parse input event. + String[] cols = event.getEvent().split(","); + String routingKey = cols[0]; + long intData = Long.parseLong(cols[1]); + long generatedIndex = Long.parseLong(cols[2]); + String generatedTimestampStr = cols[3]; + + // Process the input event and update the state. + state.sum += intData; + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedIndex), + String.format("%08d", i), + generatedTimestampStr, + processedTimestampStr, + String.format("%d", state.sum)); + + // Write the output event. + log.info("Writing message '{}' with routing key '{}' to stream {}/{}", + message, routingKey, scope, outputStreamName); + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + } + } + } + } + } + + public static void main(String[] args) throws Exception { + Master master = new Master( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getStream2Name(), + Parameters.getControllerURI(), + Parameters.getNumWorkers()); + master.run(); + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java new file mode 100644 index 00000000..9b5c2cc3 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -0,0 +1,163 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * and writing each output event to another Pravega stream. + * + * This runs only a single thread. + * Upon restart, it reprocesses the entire input stream and recreates the output stream. + * + * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} + * to view the output events. + * + * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. + */ +public class NonRecoverableSingleThreadedProcessor { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(NonRecoverableSingleThreadedProcessor.class); + + private static final int READER_TIMEOUT_MS = 2000; + + public final String scope; + public final String inputStreamName; + public final String outputStreamName; + public final URI controllerURI; + + private static class State { + long sum; + + public State(long sum) { + this.sum = sum; + } + } + + public State state; + + public NonRecoverableSingleThreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + } + + public static void main(String[] args) throws Exception { + NonRecoverableSingleThreadedProcessor processor = new NonRecoverableSingleThreadedProcessor( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getStream2Name(), + Parameters.getControllerURI()); + processor.run(); + } + + public void run() throws Exception { + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + final boolean scopeIsNew = streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + // Since we start reading the input stream from the earliest event, we must delete the output stream. + try { + streamManager.sealStream(scope, outputStreamName); + } catch (Exception e) { + if (!(e.getCause() instanceof InvalidStreamException)) { + throw e; + } + } + streamManager.deleteStream(scope, outputStreamName); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + // Create a reader group that begins at the earliest event. + final String readerGroup = UUID.randomUUID().toString().replace("-", ""); + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, inputStreamName)) + .build(); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + } + + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { + + // Initialize state. + state = new State(0); + + EventRead event; + for (int i = 0; ; i++) { + // Read input event. + try { + event = reader.readNextEvent(READER_TIMEOUT_MS); + } catch (ReinitializationRequiredException e) { + // There are certain circumstances where the reader needs to be reinitialized + log.error("Read error", e); + throw e; + } + + if (event.getEvent() != null) { + log.info("Read event '{}'", event.getEvent()); + + // Parse input event. + String[] cols = event.getEvent().split(","); + String routingKey = cols[0]; + long intData = Long.parseLong(cols[1]); + long generatedIndex = Long.parseLong(cols[2]); + String generatedTimestampStr = cols[3]; + + // Process the input event and update the state. + state.sum += intData; + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedIndex), + String.format("%08d", i), + generatedTimestampStr, + processedTimestampStr, + String.format("%d", state.sum)); + + // Write the output event. + log.info("Writing message '{}' with routing key '{}' to stream {}/{}", + message, routingKey, scope, outputStreamName); + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + } + } + } + } + +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java new file mode 100644 index 00000000..8913b21a --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java @@ -0,0 +1,66 @@ +package io.pravega.example.streamprocessing; + +import java.net.URI; +import java.nio.file.Path; +import java.nio.file.Paths; + +// All parameters will come from environment variables. This makes it easy +// to configure on Docker, Kubernetes, etc. +class Parameters { + // By default, we will connect to a standalone Pravega running on localhost. + public static URI getControllerURI() { + return URI.create(getEnvVar("PRAVEGA_CONTROLLER", "tcp://localhost:9090")); + } + + public static String getScope() { + return getEnvVar("PRAVEGA_SCOPE", "examples"); + } + + public static String getStream1Name() { + return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1"); + } + + public static String getStream2Name() { + return getEnvVar("PRAVEGA_STREAM_2", "streamprocessing2"); + } + + public static int getTargetRateEventsPerSec() { + return Integer.parseInt(getEnvVar("PRAVEGA_TARGET_RATE_EVENTS_PER_SEC", "10")); + } + + public static int getScaleFactor() { + return Integer.parseInt(getEnvVar("PRAVEGA_SCALE_FACTOR", "2")); + } + + public static int getMinNumSegments() { + return Integer.parseInt(getEnvVar("PRAVEGA_MIN_NUM_SEGMENTS", "3")); + } + + public static int getNumWorkers() { + return Integer.parseInt(getEnvVar("NUM_WORKERS", "2")); + } + + public static Path getCheckpointRootPath() { + return Paths.get(getEnvVar("CHECKPOINT_ROOT_PATH", "/tmp/checkpoints")); + } + + public static long getCheckpointPeriodMs() { + return Long.parseLong(getEnvVar("CHECKPOINT_PERIOD_MS", "3000")); + } + + public static long getCheckpointTimeoutMs() { + return Long.parseLong(getEnvVar("CHECKPOINT_TIMEOUT_MS", "10000")); + } + + public static long getTransactionTimeoutMs() { + return Long.parseLong(getEnvVar("TRANSACTION_TIMEOUT_MS", "120000")); + } + + private static String getEnvVar(String name, String defaultValue) { + String value = System.getenv(name); + if (value == null || value.isEmpty()) { + return defaultValue; + } + return value; + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md new file mode 100644 index 00000000..4143f8ca --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -0,0 +1,177 @@ +# Pravega Stream Processing Example + + +# Overview + +The examples in this directory are intended to illustrate +how exactly-once semantics can be achieved with Pravega. +To make it clear to the reader, the entirety of the stream processor is defined +in a single source file of under 500 lines. +In particular, these illustrative examples do not use Apache Flink. + +As one reads and understands this example exactly-once processor, they should +understand that enterprise-grade stream processing at scale requires +a sophisticated stream processing system like Apache Flink. +In addition to providing exactly-once processing, it also handles +windowing, aggregations, event time processing, state management, and a lot more. + +These examples include: + +- [EventGenerator](EventGenerator.java): + This application generates new events every 1 second + and writes them to a Pravega stream (referred to as stream1). + +- [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java): + This application continuously reads events from stream1, performs a stateful computation + to generate output events, and writes the output event to another + Pravega stream (referred to as stream2). + It uses the exactly-once algorithms described below. + +- [EventDebugSink](EventDebugSink.java): + This application reads events from stream2 and displays them + on the console. + + +# How to Run + +- Enable INFO level logging by editing the file [logback.xml](../../../../../resources/logback.xml). + Update it to include: + ``` + + ``` + +- Start the event generator. + ``` + cd pravega-samples + ./gradlew pravega-client-examples:startEventGenerator + ``` + + All parameters are provided as environment variables. + You can either set them in your shell (`export PRAVEGA_SCOPE=examples`) or use the below syntax. + + If you are using a non-local Pravega instance, specify the controller as follows: + ``` + PRAVEGA_CONTROLLER=tcp://pravega.example.com:9090 ./gradlew pravega-client-examples:startEventGenerator + ``` + + Multiple parameters can be specified as follows. + ``` + PRAVEGA_SCOPE=examples PRAVEGA_CONTROLLER=tcp://localhost:9090 ./gradlew pravega-client-examples:startEventGenerator + ``` + + See [Parameters.java](Parameters.java) for available parameters. + +- In another window, start the stream processor: + ``` + ./gradlew pravega-client-examples:startExactlyOnceMultithreadedProcessor + ``` + +- In another window, start the event debug sink: + ``` + ./gradlew pravega-client-examples:startEventDebugSink + ``` + +- Note: The [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java) + will automatically restart from the latest checkpoint. + However, if Pravega streams are truncated or deleted, or checkpoint files in + /tmp/checkpoints are deleted or otherwise bad, you may need to start over from + a clean system. To do so, follow these steps: + - Stop the event generator, stream processor, and event debug sink. + - Delete the contents of /tmp/checkpoints. + - Use a new scope (PRAVEGA_SCOPE) or streams (PRAVEGA_STREAM_1 and PRAVEGA_STREAM_2). + + +# Achieving At-Least-Once Semantics + +The current position in a Pravega stream is defined by a stream cut. +A stream cut is essentially a mapping from segment numbers to byte offsets +within that segment. +However, because the number of concurrent segments can dynamically +increase and decrease for a single stream, it is not possible for +a Pravega application to maintain this mapping by itself. +To achieve at-least-once semantics with Pravega, use the algorithms in this section. + +In the algorithms below, one process in a distributed system will be designated +by the user as a master. There can be any number of worker processes. + +## Checkpoint Algorithm + +- Master + - Create a checkpoint name as a new random UUID. + - Create a directory in persistent storage (e.g. NFS/HDFS) to store information + about this checkpoint. + For instance: /checkpoints/5ef9b301-2af1-4320-8028-c4cef9f39aca + - Initiate a checkpoint on the reader group used by the workers. + - Wait for all workers to complete handling the checkpoint. + - After waiting, the Pravega checkpoint object, including the + stream cuts, will be available. + - Write the Pravega checkpoint object to the checkpoint directory. + - Atomically update the file /checkpoints/latest to reference the + checkpoint name. + +- Worker + - Loop: + - Read the next event from the reader group. + Immediately after a checkpoint event, reading the next event + will notify the master that it has finished handling the checkpoint. + - If this is a checkpoint event: + - Write worker state to the checkpoint directory. + - If this is a normal event: + - Process the event and write output event(s). + +## Restart/Recovery Algorithm + +- Master: + - Read the file /checkpoints/latest to determine the directory + containing the latest successful checkpoint. + - Read the Pravega checkpoint object to determine the stream cut + at the last checkpoint. + - Create a reader group that starts reading from this stream cut. + - Launch worker processes, passing the name of the checkpoint and + the reader group. + +- Worker: + - Load worker state from the checkpoint directory. + - Create a Pravega reader that reads from the reader group. + - Process events as usual. + + +# Achieving Exactly-Once Semantics + +Exactly-once semantics can achieved by starting with at-least-once +semantics and adding write idempotence. +In Pravega, write idempotence can be achieved by using Pravega +transactions. + +Essentially, the at-least-once algorithm is modified in the following ways: + +- Each workers start its own Pravega transaction. + All output events are written to the transaction. + If a worker writes to multiple Pravega streams, it would use + a different transaction for each stream. + +- When a worker receives a checkpoint event, it flushes + (but does not commit) the transaction + and adds the transaction ID to the checkpoint directory. + +- When all workers complete handling of the checkpoint, the master + updates the checkpoint directory as before and then + commits all transactions referenced in the checkpoint directory. + +- Upon restart, any transactions referenced in the checkpoint + directory are committed if they have not yet been committed. + + Note: If any transactions fail to commit (usually due to a + transaction timeout), the stream processor will terminate + as it cannot guarantee exactly-once semantics. + To maintain correctness, recovery + from such a situation will usually require deleting the output stream + and recreating it from the beginning. + To avoid such problems, ensure that failed stream processors + are restarted before transactions timeout. + See also + [EventWriterConfig.java](https://github.com/pravega/pravega/blob/r0.4/client/src/main/java/io/pravega/client/stream/EventWriterConfig.java#L27). + + +See [ExactlyOnceMultithreadedProcessor.java](ExactlyOnceMultithreadedProcessor.java) +for details. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java new file mode 100644 index 00000000..782c905b --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java @@ -0,0 +1,279 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.io.FileOutputStream; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.*; +import java.util.stream.IntStream; + +/** + * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * and writing each output event to another Pravega stream. + */ +public class RecoverableMultithreadedProcessor { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(RecoverableMultithreadedProcessor.class); + + public static class Master implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); + + private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; + private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; + + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; + private final int numWorkers; + private final String readerGroupName; + private final ReaderGroup readerGroup; + private final ScheduledExecutorService initiateCheckpointExecutor; + private final ScheduledExecutorService performCheckpointExecutor; + private final ExecutorService workerExecutor; + private final ReaderGroupManager readerGroupManager; + private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); + private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); + private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); + private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); + + public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + this.numWorkers = numWorkers; + + // Create streams. + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + // Since we start reading the input stream from the earliest event, we must delete the output stream. + streamManager.sealStream(scope, outputStreamName); + streamManager.deleteStream(scope, outputStreamName); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + // Create a reader group manager. It must remain open to allow manual checkpoints to work. + readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); + + ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() + .disableAutomaticCheckpoints(); + + // Attempt to load the last Pravega checkpoint. + try { + String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Reading Pravega checkpoint from {}", checkpointPath); + Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); + log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + builder = builder.startFromCheckpoint(checkpoint); + } catch (Exception e) { + log.warn("Unable to load checkpoint from {}. Starting from the earliest event.", checkpointRootPath, e); + // This will create a reader group that starts from the earliest event. + builder = builder.stream(Stream.of(scope, inputStreamName)); + } + + final ReaderGroupConfig readerGroupConfig = builder.build(); + readerGroupName = UUID.randomUUID().toString().replace("-", ""); + readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); + readerGroup = readerGroupManager.getReaderGroup(readerGroupName); + + initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); + performCheckpointExecutor = Executors.newScheduledThreadPool(1); + workerExecutor = Executors.newFixedThreadPool(numWorkers); + + // Schedule periodic task to initiate checkpoints. + // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. + initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); + } + + public void run() { + IntStream.range(0, numWorkers).forEach(workerIndex -> { + Worker worker = new Worker(workerIndex, scope, readerGroupName, outputStreamName, controllerURI); + workerExecutor.submit(worker); + }); + try { + workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { + } + } + /** + * Perform a checkpoint, wait for it to complete, and write the checkpoint to the state. + */ + private void performCheckpoint() { + final String checkpointName = UUID.randomUUID().toString(); + log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); + try { + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + checkpointDirPath.toFile().mkdirs(); + + final Set onlineReaders = readerGroup.getOnlineReaders(); + log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); + log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); + CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); + log.debug("performCheckpoint: Got future."); + Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); + log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Writing Pravega checkpoint to {}", checkpointPath); + try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { + fos.write(checkpoint.toBytes().array()); + fos.flush(); + fos.getFD().sync(); + } + + Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + "tmp"); + try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { + fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); + fos.flush(); + fos.getFD().sync(); + } + Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); + + // TODO: Cleanup all other checkpoints. + } catch (final Exception e) { + log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); + // Ignore error. We will retry when we are scheduled again. + } + log.info("performCheckpoint: END: checkpointName={}", checkpointName); + } + } + + public static class Worker implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); + + private static class State { + long sum; + + public State(long sum) { + this.sum = sum; + } + } + + private static final int readerTimeoutMs = 2000; + + private final int workerIndex; + private final String scope; + private final String readerGroupName; + private final String outputStreamName; + private final URI controllerURI; + private final String readerId; + + private State state; + + public Worker(int workerIndex, String scope, String readerGroupName, String outputStreamName, URI controllerURI) { + this.workerIndex = workerIndex; + this.scope = scope; + this.readerGroupName = readerGroupName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + readerId = "worker-" + this.workerIndex; + } + + public void run() { + Thread.currentThread().setName("worker-" + workerIndex); + log.info("BEGIN"); + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + readerId, + readerGroupName, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { + + // Initialize state. + state = new State(0); + + EventRead event; + for (int i = 0; ; i++) { + // Read input event. + try { + event = reader.readNextEvent(readerTimeoutMs); + } catch (ReinitializationRequiredException e) { + // There are certain circumstances where the reader needs to be reinitialized + log.error("Read error", e); + throw new RuntimeException(e); + } + + if (event.isCheckpoint()) { + log.info("Got checkpoint {}", event.getCheckpointName()); + } + else if (event.getEvent() != null) { + log.info("Read event '{}'", event.getEvent()); + + // Parse input event. + String[] cols = event.getEvent().split(","); + String routingKey = cols[0]; + long intData = Long.parseLong(cols[1]); + long generatedIndex = Long.parseLong(cols[2]); + String generatedTimestampStr = cols[3]; + + // Process the input event and update the state. + state.sum += intData; + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedIndex), + String.format("%08d", i), + generatedTimestampStr, + processedTimestampStr, + String.format("%d", state.sum)); + + // Write the output event. + log.info("Writing message '{}' with routing key '{}' to stream {}/{}", + message, routingKey, scope, outputStreamName); + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + } + } + } + } + } + + public static void main(String[] args) throws Exception { + Master master = new Master( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getStream2Name(), + Parameters.getControllerURI(), + Parameters.getNumWorkers()); + master.run(); + } +} From 666c86eea79058e091715b6e92b316a9e2f8a70b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 16 Mar 2019 11:05:46 -0700 Subject: [PATCH 02/78] Issue 191: Moved inner classes to top level. Removed obsolete classes. Signed-off-by: Claudio Fahey --- .../ExactlyOnceMultithreadedProcessor.java | 661 +++++++----------- ...actlyOnceMultithreadedProcessorWorker.java | 163 +++++ .../NonRecoverableMultithreadedProcessor.java | 209 ------ ...NonRecoverableSingleThreadedProcessor.java | 10 +- .../RecoverableMultithreadedProcessor.java | 279 -------- .../example/streamprocessing/State.java | 20 + 6 files changed, 428 insertions(+), 914 deletions(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java index 805d5d88..7b2f342a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -1,13 +1,3 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ package io.pravega.example.streamprocessing; import io.pravega.client.ClientFactory; @@ -17,14 +7,16 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; -import java.io.*; +import java.io.FileOutputStream; +import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.nio.file.*; import java.nio.file.attribute.BasicFileAttributes; -import java.text.SimpleDateFormat; -import java.util.*; +import java.util.List; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.*; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -38,453 +30,288 @@ * * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} * to view the output events. + * + * See also {@link ExactlyOnceMultithreadedProcessorWorker}. */ -public class ExactlyOnceMultithreadedProcessor { + +public class ExactlyOnceMultithreadedProcessor implements Runnable { private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessor.class); - private static final String CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX = "pravega-transactions-worker-"; - - public static class Master implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); - - private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; - private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; - - private final String scope; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - private final int numWorkers; - private final String readerGroupName; - private final ReaderGroup readerGroup; - private final ScheduledExecutorService initiateCheckpointExecutor; - private final ScheduledExecutorService performCheckpointExecutor; - private final ExecutorService workerExecutor; - private final ReaderGroupManager readerGroupManager; - private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); - private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); - private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); - private final boolean startFromCheckpoint; - private final String startFromCheckpointName; - private final ClientFactory clientFactory; - private final EventStreamWriter writer; - - public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - this.numWorkers = numWorkers; - - ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() - .disableAutomaticCheckpoints(); - - // Load the last checkpoint. - startFromCheckpoint = latestCheckpointNamePath.toFile().exists(); - if (startFromCheckpoint) { - // Read the name of the checkpoint from the file /tmp/checkpoint/latest. - String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Reading Pravega checkpoint from {}", checkpointPath); - Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); - log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - builder = builder.startFromCheckpoint(checkpoint); - startFromCheckpointName = checkpointName; - } else { - log.warn("Checkpoint file {} not found. Starting processing from the earliest event.", checkpointRootPath); - - // Create streams. - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - // Since we are starting processing from the beginning, delete and create a new output stream. - // TODO: Should we truncate stream instead of deleting? - try { - streamManager.sealStream(scope, outputStreamName); - } catch (Exception e) { - if (!(e.getCause() instanceof InvalidStreamException)) { - throw e; - } + private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; + private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; + static final String CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX = "pravega-transactions-worker-"; + + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; + private final int numWorkers; + private final String readerGroupName; + private final ReaderGroup readerGroup; + private final ScheduledExecutorService initiateCheckpointExecutor; + private final ScheduledExecutorService performCheckpointExecutor; + private final ExecutorService workerExecutor; + private final ReaderGroupManager readerGroupManager; + private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); + private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); + private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); + private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); + private final boolean startFromCheckpoint; + private final String startFromCheckpointName; + private final ClientFactory clientFactory; + private final EventStreamWriter writer; + + public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + this.numWorkers = numWorkers; + + ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() + .disableAutomaticCheckpoints(); + + // Load the last checkpoint. + startFromCheckpoint = latestCheckpointNamePath.toFile().exists(); + if (startFromCheckpoint) { + // Read the name of the checkpoint from the file /tmp/checkpoint/latest. + String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Reading Pravega checkpoint from {}", checkpointPath); + Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); + log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + builder = builder.startFromCheckpoint(checkpoint); + startFromCheckpointName = checkpointName; + } else { + log.warn("Checkpoint file {} not found. Starting processing from the earliest event.", checkpointRootPath); + + // Create streams. + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + // Since we are starting processing from the beginning, delete and create a new output stream. + // TODO: Should we truncate stream instead of deleting? + try { + streamManager.sealStream(scope, outputStreamName); + } catch (Exception e) { + if (!(e.getCause() instanceof InvalidStreamException)) { + throw e; } - // TODO: It would be nice if deleteStream did not require sealStream to be called. - streamManager.deleteStream(scope, outputStreamName); - streamManager.createStream(scope, outputStreamName, streamConfig); } - - // Create a reader group that starts from the earliest event. - builder = builder.stream(Stream.of(scope, inputStreamName)); - startFromCheckpointName = null; + // TODO: It would be nice if deleteStream did not require sealStream to be called. + streamManager.deleteStream(scope, outputStreamName); + streamManager.createStream(scope, outputStreamName, streamConfig); } - clientFactory = ClientFactory.withScope(scope, controllerURI); - writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build()); - - // Create a reader group manager. It must remain open to allow manual checkpoints to work. - readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); - - final ReaderGroupConfig readerGroupConfig = builder.build(); - readerGroupName = UUID.randomUUID().toString().replace("-", ""); - readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); - readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - - initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); - performCheckpointExecutor = Executors.newScheduledThreadPool(1); - workerExecutor = Executors.newFixedThreadPool(numWorkers); + // Create a reader group that starts from the earliest event. + builder = builder.stream(Stream.of(scope, inputStreamName)); + startFromCheckpointName = null; } - /** - * Commit all transactions that are part of a checkpoint. - * - * @param checkpointName - */ - private void commitTransactions(String checkpointName) { - log.info("commitTransactions: BEGIN"); + clientFactory = ClientFactory.withScope(scope, controllerURI); + writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build()); - // Read the contents of all pravega-transactions-worker-XX files. - // These files contain the Pravega transaction IDs that must be committed now. - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - List txnIds = IntStream - .range(0, numWorkers) - .boxed() - .map(workerIndex -> checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + workerIndex)) - .flatMap(path -> { - try { - return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); - } catch (IOException e) { - throw new RuntimeException(e); - } - }) - .map(UUID::fromString) - .collect(Collectors.toList()); + // Create a reader group manager. It must remain open to allow manual checkpoints to work. + readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); - log.info("commitTransactions: txnIds={}", txnIds); + final ReaderGroupConfig readerGroupConfig = builder.build(); + readerGroupName = UUID.randomUUID().toString().replace("-", ""); + readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); + readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - // Initiate commit of all transactions in the checkpoint. - txnIds.parallelStream().forEach(txnId -> { - try { - Transaction transaction = writer.getTxn(txnId); - Transaction.Status status = transaction.checkStatus(); - log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); - if (status == Transaction.Status.OPEN) { - log.info("commitTransaction: committing {}", transaction.getTxnId()); - transaction.commit(); - // Note that commit may return before the transaction is committed. - // TODO: It would be nice for commit() to return a future when it becomes COMMITTED or ABORTED. - } - } catch (TxnFailedException e) { - throw new RuntimeException(e); - } - }); + initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); + performCheckpointExecutor = Executors.newScheduledThreadPool(1); + workerExecutor = Executors.newFixedThreadPool(numWorkers); + } - // Wait for commit of all transactions in the checkpoint. - txnIds.parallelStream().forEach(txnId -> { - try { - Transaction transaction = writer.getTxn(txnId); - // TODO: Is there a better way to wait for COMMITTED besides polling? - for (; ; ) { - Transaction.Status status = transaction.checkStatus(); - log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); - if (status == Transaction.Status.COMMITTED) { - log.info("commitTransaction: committed {}", transaction.getTxnId()); - break; - } else if (status == Transaction.Status.ABORTED) { - throw new RuntimeException(new TxnFailedException()); - } - Thread.sleep(100); + /** + * Commit all transactions that are part of a checkpoint. + * + * @param checkpointName + */ + private void commitTransactions(String checkpointName) { + log.info("commitTransactions: BEGIN"); + + // Read the contents of all pravega-transactions-worker-XX files. + // These files contain the Pravega transaction IDs that must be committed now. + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + List txnIds = IntStream + .range(0, numWorkers) + .boxed() + .map(workerIndex -> checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + workerIndex)) + .flatMap(path -> { + try { + return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); + } catch (IOException e) { + throw new RuntimeException(e); } - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }); + }) + .map(UUID::fromString) + .collect(Collectors.toList()); - log.info("commitTransactions: END"); - } + log.info("commitTransactions: txnIds={}", txnIds); - public void run() { - // It is possible that the checkpoint was completely written but that some or all Pravega transactions - // have not been committed. This will ensure that they are. - if (startFromCheckpoint) { - commitTransactions(startFromCheckpointName); - } - - // Schedule periodic task to initiate checkpoints. - // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. - initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); - - // Start workers. - IntStream.range(0, numWorkers).forEach(workerIndex -> { - Worker worker = new Worker(workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI); - workerExecutor.submit(worker); - }); + // Initiate commit of all transactions in the checkpoint. + txnIds.parallelStream().forEach(txnId -> { try { - workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { + Transaction transaction = writer.getTxn(txnId); + Transaction.Status status = transaction.checkStatus(); + log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); + if (status == Transaction.Status.OPEN) { + log.info("commitTransaction: committing {}", transaction.getTxnId()); + transaction.commit(); + // Note that commit may return before the transaction is committed. + // TODO: It would be nice for commit() to return a future when it becomes COMMITTED or ABORTED. + } + } catch (TxnFailedException e) { + throw new RuntimeException(e); } - } + }); - /** - * Initiate a checkpoint, wait for it to complete, and write the checkpoint to the state. - */ - private void performCheckpoint() { - final String checkpointName = UUID.randomUUID().toString(); - log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); + // Wait for commit of all transactions in the checkpoint. + txnIds.parallelStream().forEach(txnId -> { try { - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - checkpointDirPath.toFile().mkdirs(); - - final Set onlineReaders = readerGroup.getOnlineReaders(); - log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); - log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); - CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); - Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); - // At this point, all workers have received and processed the checkpoint. - log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Writing Pravega checkpoint to {}", checkpointPath); - try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { - fos.write(checkpoint.toBytes().array()); - fos.flush(); - fos.getFD().sync(); - } - - // Create "latest" file that indicates the latest checkpoint name. - // This file must be updated atomically. - Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + ".tmp"); - try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { - fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); - fos.flush(); - fos.getFD().sync(); + Transaction transaction = writer.getTxn(txnId); + // TODO: Is there a better way to wait for COMMITTED besides polling? + for (; ; ) { + Transaction.Status status = transaction.checkStatus(); + log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); + if (status == Transaction.Status.COMMITTED) { + log.info("commitTransaction: committed {}", transaction.getTxnId()); + break; + } else if (status == Transaction.Status.ABORTED) { + throw new RuntimeException(new TxnFailedException()); + } + Thread.sleep(100); } - Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); - - // Read list of TxnIds from checkpoint directory written by all workers and commit all transactions. - commitTransactions(checkpointName); - - cleanCheckpointDirectory(checkpointDirPath); - } catch (final Exception e) { - log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); - // Ignore error. We will retry when we are scheduled again. + } catch (InterruptedException e) { + throw new RuntimeException(e); } - log.info("performCheckpoint: END: checkpointName={}", checkpointName); - } + }); - /** - * Delete everything in the checkpoint root path (/tmp/checkpoint) except the "latest" file and - * the latest checkpoint directory. - * - * @param keepCheckpointDirPath The latest checkpoint directory which will not be deleted. - */ - private void cleanCheckpointDirectory(Path keepCheckpointDirPath) { - try { - Files.walkFileTree(checkpointRootPath, - new SimpleFileVisitor() { - @Override - public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes basicFileAttributes) throws IOException { - if (dir.equals(keepCheckpointDirPath)) { - return FileVisitResult.SKIP_SUBTREE; - } - return FileVisitResult.CONTINUE; - } + log.info("commitTransactions: END"); + } - @Override - public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { - if (!dir.equals(checkpointRootPath)) { - Files.delete(dir); - } - return FileVisitResult.CONTINUE; - } + public void run() { + // It is possible that the checkpoint was completely written but that some or all Pravega transactions + // have not been committed. This will ensure that they are. + if (startFromCheckpoint) { + commitTransactions(startFromCheckpointName); + } - @Override - public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { - if (!file.equals(latestCheckpointNamePath)) { - Files.delete(file); - } - return FileVisitResult.CONTINUE; - } - }); - } catch (IOException e) { - log.warn("cleanCheckpointDirectory", e); - } + // Schedule periodic task to initiate checkpoints. + // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. + initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); + + // Start workers. + IntStream.range(0, numWorkers).forEach(workerIndex -> { + ExactlyOnceMultithreadedProcessorWorker worker = new ExactlyOnceMultithreadedProcessorWorker(workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI); + workerExecutor.submit(worker); + }); + try { + workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + } catch (InterruptedException e) { } } - public static class Worker implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); - - private static final String STATE_FILE_NAME_PREFIX = "state-worker-"; - private static final int READER_TIMEOUT_MS = 2000; - - private static class State implements Serializable { - private static final long serialVersionUID = -275148988691911596L; - - long sum; - - public State() { - this.sum = 0; + /** + * Initiate a checkpoint, wait for it to complete, and write the checkpoint to the state. + */ + private void performCheckpoint() { + final String checkpointName = UUID.randomUUID().toString(); + log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); + try { + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + checkpointDirPath.toFile().mkdirs(); + + final Set onlineReaders = readerGroup.getOnlineReaders(); + log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); + log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); + CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); + Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); + // At this point, all workers have received and processed the checkpoint. + log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); + + Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); + log.info("Writing Pravega checkpoint to {}", checkpointPath); + try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { + fos.write(checkpoint.toBytes().array()); + fos.flush(); + fos.getFD().sync(); } - @Override - public String toString() { - return "State{" + - "sum=" + sum + - '}'; + // Create "latest" file that indicates the latest checkpoint name. + // This file must be updated atomically. + Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + ".tmp"); + try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { + fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); + fos.flush(); + fos.getFD().sync(); } - } + Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); - private final int workerIndex; - private final String scope; - private final String readerGroupName; - private final boolean startFromCheckpoint; - private final String startFromCheckpointName; - private final String outputStreamName; - private final URI controllerURI; - private final String readerId; - private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - - private State state; - - public Worker(int workerIndex, String scope, String readerGroupName, String startFromCheckpointName, String outputStreamName, URI controllerURI) { - this.workerIndex = workerIndex; - this.scope = scope; - this.readerGroupName = readerGroupName; - this.startFromCheckpointName = startFromCheckpointName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - readerId = "worker-" + this.workerIndex; - startFromCheckpoint = startFromCheckpointName != null; - } + // Read list of TxnIds from checkpoint directory written by all workers and commit all transactions. + commitTransactions(checkpointName); - public void run() { - Thread.currentThread().setName("worker-" + workerIndex); - log.info("BEGIN"); - - try { - // Load state from checkpoint. - if (startFromCheckpoint) { - Path checkpointDirPath = checkpointRootPath.resolve(startFromCheckpointName); - Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); - log.info("statePath={}", statePath.toString()); - try (FileInputStream fis = new FileInputStream(statePath.toString()); - ObjectInputStream ois = new ObjectInputStream(fis)) { - state = (State) ois.readObject(); - } - log.info("Loaded state {} from {}", state, statePath); - } else { - log.info("Initializing with new state"); - state = new State(); - } + cleanCheckpointDirectory(checkpointDirPath); + } catch (final Exception e) { + log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); + // Ignore error. We will retry when we are scheduled again. + } + log.info("performCheckpoint: END: checkpointName={}", checkpointName); + } - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); - EventStreamReader reader = clientFactory.createReader( - readerId, - readerGroupName, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder() - .transactionTimeoutTime(Parameters.getTransactionTimeoutMs()) - .build())) { - - Transaction transaction = null; - long eventCounter = 0; - - for (; ; ) { - // Read input event. - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - log.debug("readEvents: eventRead={}", eventRead); - - if (eventRead.isCheckpoint()) { - // Note that next call readNextEvent will indicate to Pravega that we are done with the checkpoint. - String checkpointName = eventRead.getCheckpointName(); - log.info("Got checkpoint {}", eventRead.getCheckpointName()); - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - Path transactionIdFilePath = checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + this.workerIndex); - - // Must ensure that txnId is persisted to latest state before committing transaction! - // Do not commit transaction here. Instead write TxnId to checkpoint directory. Master will read all TxnIds and commit transactions. - - String transactionIds = ""; - if (transaction != null) { - transaction.flush(); - transactionIds = transaction.getTxnId().toString(); - transaction = null; + /** + * Delete everything in the checkpoint root path (/tmp/checkpoint) except the "latest" file and + * the latest checkpoint directory. + * + * @param keepCheckpointDirPath The latest checkpoint directory which will not be deleted. + */ + private void cleanCheckpointDirectory(Path keepCheckpointDirPath) { + try { + Files.walkFileTree(checkpointRootPath, + new SimpleFileVisitor() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes basicFileAttributes) throws IOException { + if (dir.equals(keepCheckpointDirPath)) { + return FileVisitResult.SKIP_SUBTREE; } - Files.write(transactionIdFilePath, transactionIds.getBytes(StandardCharsets.UTF_8)); - - // Write state to checkpoint directory - Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); - log.info("statePath={}", statePath.toString()); - try (FileOutputStream fos = new FileOutputStream(statePath.toString()); - ObjectOutputStream oos = new ObjectOutputStream(fos)) { - oos.writeObject(state); - oos.flush(); - fos.getFD().sync(); - } - - } else if (eventRead.getEvent() != null) { - eventCounter++; - log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); + return FileVisitResult.CONTINUE; + } - if (transaction == null) { - transaction = writer.beginTxn(); + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + if (!dir.equals(checkpointRootPath)) { + Files.delete(dir); } + return FileVisitResult.CONTINUE; + } - // Parse input event. - String[] cols = eventRead.getEvent().split(","); - long generatedEventCounter = Long.parseLong(cols[0]); - String routingKey = cols[1]; - long intData = Long.parseLong(cols[2]); - long generatedSum = Long.parseLong(cols[3]); - String generatedTimestampStr = cols[4]; - - // Process the input event and update the state. - state.sum += intData; - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - String.format("%06d", generatedEventCounter), - String.format("%06d", eventCounter), - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedSum), - String.format("%08d", state.sum), - String.format("%03d", workerIndex), - generatedTimestampStr, - processedTimestampStr, - transaction.getTxnId().toString()); - - // Write the output event. - log.info("eventCounter={}, event={}", - String.format("%06d", eventCounter), - message); - transaction.writeEvent(routingKey, message); + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + if (!file.equals(latestCheckpointNamePath)) { + Files.delete(file); + } + return FileVisitResult.CONTINUE; } - } - } - } catch (Exception e) { - log.error("Fatal Error", e); - System.exit(1); - } + }); + } catch (IOException e) { + log.warn("cleanCheckpointDirectory", e); } } public static void main(String[] args) throws Exception { - Master master = new Master( + ExactlyOnceMultithreadedProcessor master = new ExactlyOnceMultithreadedProcessor( Parameters.getScope(), Parameters.getStream1Name(), Parameters.getStream2Name(), diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java new file mode 100644 index 00000000..80e3a7a3 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java @@ -0,0 +1,163 @@ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientFactory; +import io.pravega.client.stream.*; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.text.SimpleDateFormat; +import java.util.Date; + +/** + * See {@link ExactlyOnceMultithreadedProcessor}. + */ +public class ExactlyOnceMultithreadedProcessorWorker implements Runnable { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessorWorker.class); + + private static final String STATE_FILE_NAME_PREFIX = "state-worker-"; + private static final int READER_TIMEOUT_MS = 2000; + + private final int workerIndex; + private final String scope; + private final String readerGroupName; + private final boolean startFromCheckpoint; + private final String startFromCheckpointName; + private final String outputStreamName; + private final URI controllerURI; + private final String readerId; + private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); + + private State state; + + public ExactlyOnceMultithreadedProcessorWorker(int workerIndex, String scope, String readerGroupName, String startFromCheckpointName, String outputStreamName, URI controllerURI) { + this.workerIndex = workerIndex; + this.scope = scope; + this.readerGroupName = readerGroupName; + this.startFromCheckpointName = startFromCheckpointName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + readerId = "worker-" + this.workerIndex; + startFromCheckpoint = startFromCheckpointName != null; + } + + public void run() { + Thread.currentThread().setName("worker-" + workerIndex); + log.info("BEGIN"); + + try { + // Load state from checkpoint. + if (startFromCheckpoint) { + Path checkpointDirPath = checkpointRootPath.resolve(startFromCheckpointName); + Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); + log.info("statePath={}", statePath.toString()); + try (FileInputStream fis = new FileInputStream(statePath.toString()); + ObjectInputStream ois = new ObjectInputStream(fis)) { + state = (State) ois.readObject(); + } + log.info("Loaded state {} from {}", state, statePath); + } else { + log.info("Initializing with new state"); + state = new State(); + } + + try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + EventStreamReader reader = clientFactory.createReader( + readerId, + readerGroupName, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder() + .transactionTimeoutTime(Parameters.getTransactionTimeoutMs()) + .build())) { + + Transaction transaction = null; + long eventCounter = 0; + + for (; ; ) { + // Read input event. + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + log.debug("readEvents: eventRead={}", eventRead); + + if (eventRead.isCheckpoint()) { + // Note that next call readNextEvent will indicate to Pravega that we are done with the checkpoint. + String checkpointName = eventRead.getCheckpointName(); + log.info("Got checkpoint {}", eventRead.getCheckpointName()); + Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); + Path transactionIdFilePath = checkpointDirPath.resolve(ExactlyOnceMultithreadedProcessor.CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + this.workerIndex); + + // Must ensure that txnId is persisted to latest state before committing transaction! + // Do not commit transaction here. Instead write TxnId to checkpoint directory. Master will read all TxnIds and commit transactions. + + String transactionIds = ""; + if (transaction != null) { + transaction.flush(); + transactionIds = transaction.getTxnId().toString(); + transaction = null; + } + Files.write(transactionIdFilePath, transactionIds.getBytes(StandardCharsets.UTF_8)); + + // Write state to checkpoint directory + Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); + log.info("statePath={}", statePath.toString()); + try (FileOutputStream fos = new FileOutputStream(statePath.toString()); + ObjectOutputStream oos = new ObjectOutputStream(fos)) { + oos.writeObject(state); + oos.flush(); + fos.getFD().sync(); + } + + } else if (eventRead.getEvent() != null) { + eventCounter++; + log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); + + if (transaction == null) { + transaction = writer.beginTxn(); + } + + // Parse input event. + String[] cols = eventRead.getEvent().split(","); + long generatedEventCounter = Long.parseLong(cols[0]); + String routingKey = cols[1]; + long intData = Long.parseLong(cols[2]); + long generatedSum = Long.parseLong(cols[3]); + String generatedTimestampStr = cols[4]; + + // Process the input event and update the state. + state.sum += intData; + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + String.format("%06d", generatedEventCounter), + String.format("%06d", eventCounter), + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedSum), + String.format("%08d", state.sum), + String.format("%03d", workerIndex), + generatedTimestampStr, + processedTimestampStr, + transaction.getTxnId().toString()); + + // Write the output event. + log.info("eventCounter={}, event={}", + String.format("%06d", eventCounter), + message); + transaction.writeEvent(routingKey, message); + } + } + } + } catch (Exception e) { + log.error("Fatal Error", e); + System.exit(1); + } + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java deleted file mode 100644 index 03763f3a..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableMultithreadedProcessor.java +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ -package io.pravega.example.streamprocessing; - -import io.pravega.client.ClientFactory; -import io.pravega.client.admin.ReaderGroupManager; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.*; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import org.slf4j.LoggerFactory; - -import java.net.URI; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.UUID; -import java.util.concurrent.*; -import java.util.stream.IntStream; - -/** - * A simple example that demonstrates reading events from a Pravega stream, processing each event, - * and writing each output event to another Pravega stream. - * - * This supports multiple worker threads. - * Upon restart, it reprocesses the entire input stream and recreates the output stream. - * - * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} - * to view the output events. - * - * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. - */ -public class NonRecoverableMultithreadedProcessor { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(NonRecoverableMultithreadedProcessor.class); - - public static class Master implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); - - private final String scope; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - private final int numWorkers; - private final String readerGroupName; - private final ReaderGroup readerGroup; - private final ExecutorService workerExecutor; - - public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - this.numWorkers = numWorkers; - - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - // Since we start reading the input stream from the earliest event, we must delete the output stream. - try { - streamManager.sealStream(scope, outputStreamName); - } catch (Exception e) { - if (!(e.getCause() instanceof InvalidStreamException)) { - throw e; - } - } - streamManager.deleteStream(scope, outputStreamName); - streamManager.createStream(scope, outputStreamName, streamConfig); - } - - // Create a reader group that begins at the earliest event. - final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() - .stream(Stream.of(scope, inputStreamName)) - .build(); - readerGroupName = UUID.randomUUID().toString().replace("-", ""); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { - readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); - readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - } - - workerExecutor = Executors.newFixedThreadPool(numWorkers); - } - - public void run() { - IntStream.range(0, numWorkers).forEach(workerIndex -> { - Worker worker = new Worker(workerIndex, scope, readerGroupName, outputStreamName, controllerURI); - workerExecutor.submit(worker); - }); - try { - workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { - } - } - } - - public static class Worker implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); - - private static class State { - long sum; - - public State(long sum) { - this.sum = sum; - } - } - - private static final int readerTimeoutMs = 2000; - - private final int workerIndex; - private final String scope; - private final String readerGroupName; - private final String outputStreamName; - private final URI controllerURI; - private final String readerId; - - private State state; - - public Worker(int workerIndex, String scope, String readerGroupName, String outputStreamName, URI controllerURI) { - this.workerIndex = workerIndex; - this.scope = scope; - this.readerGroupName = readerGroupName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - readerId = "reader" + this.workerIndex; - } - - public void run() { - Thread.currentThread().setName("worker-" + workerIndex); - log.info("BEGIN"); - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); - EventStreamReader reader = clientFactory.createReader( - readerId, - readerGroupName, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build())) { - - // Initialize state. - state = new State(0); - - EventRead event; - for (int i = 0; ; i++) { - // Read input event. - try { - event = reader.readNextEvent(readerTimeoutMs); - } catch (ReinitializationRequiredException e) { - // There are certain circumstances where the reader needs to be reinitialized - log.error("Read error", e); - throw new RuntimeException(e); - } - - if (event.getEvent() != null) { - log.info("Read event '{}'", event.getEvent()); - - // Parse input event. - String[] cols = event.getEvent().split(","); - String routingKey = cols[0]; - long intData = Long.parseLong(cols[1]); - long generatedIndex = Long.parseLong(cols[2]); - String generatedTimestampStr = cols[3]; - - // Process the input event and update the state. - state.sum += intData; - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedIndex), - String.format("%08d", i), - generatedTimestampStr, - processedTimestampStr, - String.format("%d", state.sum)); - - // Write the output event. - log.info("Writing message '{}' with routing key '{}' to stream {}/{}", - message, routingKey, scope, outputStreamName); - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); - } - } - } - } - } - - public static void main(String[] args) throws Exception { - Master master = new Master( - Parameters.getScope(), - Parameters.getStream1Name(), - Parameters.getStream2Name(), - Parameters.getControllerURI(), - Parameters.getNumWorkers()); - master.run(); - } -} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java index 9b5c2cc3..1a97498b 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -45,14 +45,6 @@ public class NonRecoverableSingleThreadedProcessor { public final String outputStreamName; public final URI controllerURI; - private static class State { - long sum; - - public State(long sum) { - this.sum = sum; - } - } - public State state; public NonRecoverableSingleThreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { @@ -114,7 +106,7 @@ public void run() throws Exception { EventWriterConfig.builder().build())) { // Initialize state. - state = new State(0); + state = new State(); EventRead event; for (int i = 0; ; i++) { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java deleted file mode 100644 index 782c905b..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/RecoverableMultithreadedProcessor.java +++ /dev/null @@ -1,279 +0,0 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ -package io.pravega.example.streamprocessing; - -import io.pravega.client.ClientFactory; -import io.pravega.client.admin.ReaderGroupManager; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.*; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import org.slf4j.LoggerFactory; - -import java.io.FileOutputStream; -import java.net.URI; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardCopyOption; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.*; -import java.util.stream.IntStream; - -/** - * A simple example that demonstrates reading events from a Pravega stream, processing each event, - * and writing each output event to another Pravega stream. - */ -public class RecoverableMultithreadedProcessor { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(RecoverableMultithreadedProcessor.class); - - public static class Master implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Master.class); - - private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; - private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; - - private final String scope; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - private final int numWorkers; - private final String readerGroupName; - private final ReaderGroup readerGroup; - private final ScheduledExecutorService initiateCheckpointExecutor; - private final ScheduledExecutorService performCheckpointExecutor; - private final ExecutorService workerExecutor; - private final ReaderGroupManager readerGroupManager; - private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); - private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); - private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); - - public Master(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - this.numWorkers = numWorkers; - - // Create streams. - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - // Since we start reading the input stream from the earliest event, we must delete the output stream. - streamManager.sealStream(scope, outputStreamName); - streamManager.deleteStream(scope, outputStreamName); - streamManager.createStream(scope, outputStreamName, streamConfig); - } - - // Create a reader group manager. It must remain open to allow manual checkpoints to work. - readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); - - ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() - .disableAutomaticCheckpoints(); - - // Attempt to load the last Pravega checkpoint. - try { - String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Reading Pravega checkpoint from {}", checkpointPath); - Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); - log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - builder = builder.startFromCheckpoint(checkpoint); - } catch (Exception e) { - log.warn("Unable to load checkpoint from {}. Starting from the earliest event.", checkpointRootPath, e); - // This will create a reader group that starts from the earliest event. - builder = builder.stream(Stream.of(scope, inputStreamName)); - } - - final ReaderGroupConfig readerGroupConfig = builder.build(); - readerGroupName = UUID.randomUUID().toString().replace("-", ""); - readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); - readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - - initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); - performCheckpointExecutor = Executors.newScheduledThreadPool(1); - workerExecutor = Executors.newFixedThreadPool(numWorkers); - - // Schedule periodic task to initiate checkpoints. - // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. - initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); - } - - public void run() { - IntStream.range(0, numWorkers).forEach(workerIndex -> { - Worker worker = new Worker(workerIndex, scope, readerGroupName, outputStreamName, controllerURI); - workerExecutor.submit(worker); - }); - try { - workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { - } - } - /** - * Perform a checkpoint, wait for it to complete, and write the checkpoint to the state. - */ - private void performCheckpoint() { - final String checkpointName = UUID.randomUUID().toString(); - log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); - try { - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - checkpointDirPath.toFile().mkdirs(); - - final Set onlineReaders = readerGroup.getOnlineReaders(); - log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); - log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); - CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); - log.debug("performCheckpoint: Got future."); - Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); - log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Writing Pravega checkpoint to {}", checkpointPath); - try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { - fos.write(checkpoint.toBytes().array()); - fos.flush(); - fos.getFD().sync(); - } - - Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + "tmp"); - try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { - fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); - fos.flush(); - fos.getFD().sync(); - } - Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); - - // TODO: Cleanup all other checkpoints. - } catch (final Exception e) { - log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); - // Ignore error. We will retry when we are scheduled again. - } - log.info("performCheckpoint: END: checkpointName={}", checkpointName); - } - } - - public static class Worker implements Runnable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(Worker.class); - - private static class State { - long sum; - - public State(long sum) { - this.sum = sum; - } - } - - private static final int readerTimeoutMs = 2000; - - private final int workerIndex; - private final String scope; - private final String readerGroupName; - private final String outputStreamName; - private final URI controllerURI; - private final String readerId; - - private State state; - - public Worker(int workerIndex, String scope, String readerGroupName, String outputStreamName, URI controllerURI) { - this.workerIndex = workerIndex; - this.scope = scope; - this.readerGroupName = readerGroupName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - readerId = "worker-" + this.workerIndex; - } - - public void run() { - Thread.currentThread().setName("worker-" + workerIndex); - log.info("BEGIN"); - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); - EventStreamReader reader = clientFactory.createReader( - readerId, - readerGroupName, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build())) { - - // Initialize state. - state = new State(0); - - EventRead event; - for (int i = 0; ; i++) { - // Read input event. - try { - event = reader.readNextEvent(readerTimeoutMs); - } catch (ReinitializationRequiredException e) { - // There are certain circumstances where the reader needs to be reinitialized - log.error("Read error", e); - throw new RuntimeException(e); - } - - if (event.isCheckpoint()) { - log.info("Got checkpoint {}", event.getCheckpointName()); - } - else if (event.getEvent() != null) { - log.info("Read event '{}'", event.getEvent()); - - // Parse input event. - String[] cols = event.getEvent().split(","); - String routingKey = cols[0]; - long intData = Long.parseLong(cols[1]); - long generatedIndex = Long.parseLong(cols[2]); - String generatedTimestampStr = cols[3]; - - // Process the input event and update the state. - state.sum += intData; - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedIndex), - String.format("%08d", i), - generatedTimestampStr, - processedTimestampStr, - String.format("%d", state.sum)); - - // Write the output event. - log.info("Writing message '{}' with routing key '{}' to stream {}/{}", - message, routingKey, scope, outputStreamName); - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); - } - } - } - } - } - - public static void main(String[] args) throws Exception { - Master master = new Master( - Parameters.getScope(), - Parameters.getStream1Name(), - Parameters.getStream2Name(), - Parameters.getControllerURI(), - Parameters.getNumWorkers()); - master.run(); - } -} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java new file mode 100644 index 00000000..b3836b81 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java @@ -0,0 +1,20 @@ +package io.pravega.example.streamprocessing; + +import java.io.Serializable; + +class State implements Serializable { + private static final long serialVersionUID = -275148988691911596L; + + long sum; + + public State() { + this.sum = 0; + } + + @Override + public String toString() { + return "State{" + + "sum=" + sum + + '}'; + } +} From eded36163208ffe1302149922c80830c6c0195a4 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 16 Mar 2019 13:05:47 -0700 Subject: [PATCH 03/78] Issue 191: Various updates. This PR is still incomplete. - Removed deleteStream calls. - No longer waiting for transactions to go into COMMIT state after commit() call. - Now using Callable instead of Runnable. - Improved shutdown cleanup. Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 8 ++ .../ExactlyOnceMultithreadedProcessor.java | 74 +++++++------------ ...actlyOnceMultithreadedProcessorWorker.java | 27 +++++-- ...NonRecoverableSingleThreadedProcessor.java | 11 +-- 4 files changed, 56 insertions(+), 64 deletions(-) diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index d73cb1da..9fe5626f 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -187,6 +187,14 @@ task startEventGenerator(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath } +task scriptExactlyOnceMultithreadedProcessor(type: CreateStartScripts) { + outputDir = file('build/scripts') + mainClassName = 'io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor' + applicationName = 'ExactlyOnceMultithreadedProcessor' + defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] + classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath +} + task startExactlyOnceMultithreadedProcessor(type: JavaExec) { main = "io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor" classpath = sourceSets.main.runtimeClasspath diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java index 7b2f342a..4da12750 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -14,6 +14,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.*; import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.UUID; @@ -34,7 +35,7 @@ * See also {@link ExactlyOnceMultithreadedProcessorWorker}. */ -public class ExactlyOnceMultithreadedProcessor implements Runnable { +public class ExactlyOnceMultithreadedProcessor implements Callable { private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessor.class); private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; @@ -96,17 +97,6 @@ public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, S Parameters.getMinNumSegments())) .build(); streamManager.createStream(scope, inputStreamName, streamConfig); - // Since we are starting processing from the beginning, delete and create a new output stream. - // TODO: Should we truncate stream instead of deleting? - try { - streamManager.sealStream(scope, outputStreamName); - } catch (Exception e) { - if (!(e.getCause() instanceof InvalidStreamException)) { - throw e; - } - } - // TODO: It would be nice if deleteStream did not require sealStream to be called. - streamManager.deleteStream(scope, outputStreamName); streamManager.createStream(scope, outputStreamName, streamConfig); } @@ -147,8 +137,10 @@ private void commitTransactions(String checkpointName) { Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); List txnIds = IntStream .range(0, numWorkers) + .parallel() .boxed() .map(workerIndex -> checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + workerIndex)) + .filter(Files::exists) .flatMap(path -> { try { return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); @@ -161,7 +153,7 @@ private void commitTransactions(String checkpointName) { log.info("commitTransactions: txnIds={}", txnIds); - // Initiate commit of all transactions in the checkpoint. + // Commit all transactions that have been opened and flushed since the last checkpoint. txnIds.parallelStream().forEach(txnId -> { try { Transaction transaction = writer.getTxn(txnId); @@ -170,39 +162,17 @@ private void commitTransactions(String checkpointName) { if (status == Transaction.Status.OPEN) { log.info("commitTransaction: committing {}", transaction.getTxnId()); transaction.commit(); - // Note that commit may return before the transaction is committed. - // TODO: It would be nice for commit() to return a future when it becomes COMMITTED or ABORTED. + log.info("commitTransaction: committed {}", transaction.getTxnId()); } } catch (TxnFailedException e) { throw new RuntimeException(e); } }); - // Wait for commit of all transactions in the checkpoint. - txnIds.parallelStream().forEach(txnId -> { - try { - Transaction transaction = writer.getTxn(txnId); - // TODO: Is there a better way to wait for COMMITTED besides polling? - for (; ; ) { - Transaction.Status status = transaction.checkStatus(); - log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); - if (status == Transaction.Status.COMMITTED) { - log.info("commitTransaction: committed {}", transaction.getTxnId()); - break; - } else if (status == Transaction.Status.ABORTED) { - throw new RuntimeException(new TxnFailedException()); - } - Thread.sleep(100); - } - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }); - log.info("commitTransactions: END"); } - public void run() { + public Void call() throws InterruptedException { // It is possible that the checkpoint was completely written but that some or all Pravega transactions // have not been committed. This will ensure that they are. if (startFromCheckpoint) { @@ -213,15 +183,23 @@ public void run() { // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); - // Start workers. - IntStream.range(0, numWorkers).forEach(workerIndex -> { - ExactlyOnceMultithreadedProcessorWorker worker = new ExactlyOnceMultithreadedProcessorWorker(workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI); - workerExecutor.submit(worker); - }); - try { - workerExecutor.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - } catch (InterruptedException e) { - } + // Start workers and wait for all of them to terminate. + workerExecutor.invokeAll( + IntStream.range(0, numWorkers).boxed().map( + workerIndex -> new ExactlyOnceMultithreadedProcessorWorker( + workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI) + ).collect(Collectors.toList())); + + log.info("call: shutting down"); + workerExecutor.shutdownNow(); + performCheckpointExecutor.shutdownNow(); + initiateCheckpointExecutor.shutdownNow(); + readerGroup.close(); + readerGroupManager.close(); + writer.close(); + clientFactory.close(); + log.info("call: END"); + return null; } /** @@ -265,7 +243,7 @@ private void performCheckpoint() { cleanCheckpointDirectory(checkpointDirPath); } catch (final Exception e) { - log.warn("performCheckpoint: timed out waiting for checkpoint to complete", e); + log.warn("performCheckpoint: Error performing checkpoint", e); // Ignore error. We will retry when we are scheduled again. } log.info("performCheckpoint: END: checkpointName={}", checkpointName); @@ -317,6 +295,6 @@ public static void main(String[] args) throws Exception { Parameters.getStream2Name(), Parameters.getControllerURI(), Parameters.getNumWorkers()); - master.run(); + master.call(); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java index 80e3a7a3..c30d6559 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java @@ -12,11 +12,12 @@ import java.nio.file.Path; import java.text.SimpleDateFormat; import java.util.Date; +import java.util.concurrent.Callable; /** * See {@link ExactlyOnceMultithreadedProcessor}. */ -public class ExactlyOnceMultithreadedProcessorWorker implements Runnable { +public class ExactlyOnceMultithreadedProcessorWorker implements Callable { private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessorWorker.class); private static final String STATE_FILE_NAME_PREFIX = "state-worker-"; @@ -45,7 +46,7 @@ public ExactlyOnceMultithreadedProcessorWorker(int workerIndex, String scope, St startFromCheckpoint = startFromCheckpointName != null; } - public void run() { + public Void call() { Thread.currentThread().setName("worker-" + workerIndex); log.info("BEGIN"); @@ -55,11 +56,19 @@ public void run() { Path checkpointDirPath = checkpointRootPath.resolve(startFromCheckpointName); Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); log.info("statePath={}", statePath.toString()); - try (FileInputStream fis = new FileInputStream(statePath.toString()); - ObjectInputStream ois = new ObjectInputStream(fis)) { - state = (State) ois.readObject(); + if (Files.exists(statePath)) { + try (FileInputStream fis = new FileInputStream(statePath.toString()); + ObjectInputStream ois = new ObjectInputStream(fis)) { + state = (State) ois.readObject(); + } + log.info("Loaded state {} from {}", state, statePath); + } else { + log.info("Initializing with new state"); + // TODO: This is the wrong behavior. + // In the event that a single worker dies, the segments that it was reading will be automatically assigned to another worker. However, this code does not transfer the state between workers, leaving the state inconsistent. + state = new State(); } - log.info("Loaded state {} from {}", state, statePath); + } else { log.info("Initializing with new state"); state = new State(); @@ -134,6 +143,8 @@ public void run() { state.sum += intData; String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + if (eventCounter == 10) return null; + // Build the output event. String message = String.join(",", String.format("%06d", generatedEventCounter), @@ -157,7 +168,11 @@ public void run() { } } catch (Exception e) { log.error("Fatal Error", e); + // We don't handle incremental recovery of a single failed worker. + // Stop the entire process (master and all workers). + // When it is restarted, recovery of all workers will begin. System.exit(1); } + return null; } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java index 1a97498b..c67cc729 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -65,7 +65,7 @@ public static void main(String[] args) throws Exception { public void run() throws Exception { try (StreamManager streamManager = StreamManager.create(controllerURI)) { - final boolean scopeIsNew = streamManager.createScope(scope); + streamManager.createScope(scope); StreamConfiguration streamConfig = StreamConfiguration.builder() .scalingPolicy(ScalingPolicy.byEventRate( Parameters.getTargetRateEventsPerSec(), @@ -73,15 +73,6 @@ public void run() throws Exception { Parameters.getMinNumSegments())) .build(); streamManager.createStream(scope, inputStreamName, streamConfig); - // Since we start reading the input stream from the earliest event, we must delete the output stream. - try { - streamManager.sealStream(scope, outputStreamName); - } catch (Exception e) { - if (!(e.getCause() instanceof InvalidStreamException)) { - throw e; - } - } - streamManager.deleteStream(scope, outputStreamName); streamManager.createStream(scope, outputStreamName, streamConfig); } From 13c3931dbafd5936ec77aeefa65746f433c2a2b0 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 16 Mar 2019 21:23:20 -0700 Subject: [PATCH 04/78] Issue 191: Processor is now stateless to avoid complications with managing state. - Added additional startup scripts. - Updated default parameters. - Updated documentation. Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 18 +++++- .../ExactlyOnceMultithreadedProcessor.java | 2 +- ...actlyOnceMultithreadedProcessorWorker.java | 45 +------------ ...NonRecoverableSingleThreadedProcessor.java | 64 +++++++++---------- .../example/streamprocessing/Parameters.java | 2 +- .../example/streamprocessing/README.md | 14 ++-- .../example/streamprocessing/State.java | 20 ------ 7 files changed, 56 insertions(+), 109 deletions(-) delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 9fe5626f..8b822882 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -182,6 +182,14 @@ task startSecureBatchReader(type: JavaExec) { } } +task scriptEventGenerator(type: CreateStartScripts) { + outputDir = file('build/scripts') + mainClassName = 'io.pravega.example.streamprocessing.EventGenerator' + applicationName = 'eventGenerator' + defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] + classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath +} + task startEventGenerator(type: JavaExec) { main = "io.pravega.example.streamprocessing.EventGenerator" classpath = sourceSets.main.runtimeClasspath @@ -190,7 +198,7 @@ task startEventGenerator(type: JavaExec) { task scriptExactlyOnceMultithreadedProcessor(type: CreateStartScripts) { outputDir = file('build/scripts') mainClassName = 'io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor' - applicationName = 'ExactlyOnceMultithreadedProcessor' + applicationName = 'exactlyOnceMultithreadedProcessor' defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath } @@ -200,6 +208,14 @@ task startExactlyOnceMultithreadedProcessor(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath } +task scriptEventDebugSink(type: CreateStartScripts) { + outputDir = file('build/scripts') + mainClassName = 'io.pravega.example.streamprocessing.EventDebugSink' + applicationName = 'eventDebugSink' + defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] + classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath +} + task startEventDebugSink(type: JavaExec) { main = "io.pravega.example.streamprocessing.EventDebugSink" classpath = sourceSets.main.runtimeClasspath diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java index 4da12750..9786c55a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -14,7 +14,6 @@ import java.nio.charset.StandardCharsets; import java.nio.file.*; import java.nio.file.attribute.BasicFileAttributes; -import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.UUID; @@ -28,6 +27,7 @@ * * This supports multiple worker threads. * Upon restart, it restarts from the last successful checkpoint and guarantees exactly-once semantics. + * Workers must be stateless. * * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} * to view the output events. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java index c30d6559..0654f26b 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java @@ -5,7 +5,6 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; -import java.io.*; import java.net.URI; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -33,8 +32,6 @@ public class ExactlyOnceMultithreadedProcessorWorker implements Callable { private final String readerId; private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - private State state; - public ExactlyOnceMultithreadedProcessorWorker(int workerIndex, String scope, String readerGroupName, String startFromCheckpointName, String outputStreamName, URI controllerURI) { this.workerIndex = workerIndex; this.scope = scope; @@ -51,29 +48,6 @@ public Void call() { log.info("BEGIN"); try { - // Load state from checkpoint. - if (startFromCheckpoint) { - Path checkpointDirPath = checkpointRootPath.resolve(startFromCheckpointName); - Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); - log.info("statePath={}", statePath.toString()); - if (Files.exists(statePath)) { - try (FileInputStream fis = new FileInputStream(statePath.toString()); - ObjectInputStream ois = new ObjectInputStream(fis)) { - state = (State) ois.readObject(); - } - log.info("Loaded state {} from {}", state, statePath); - } else { - log.info("Initializing with new state"); - // TODO: This is the wrong behavior. - // In the event that a single worker dies, the segments that it was reading will be automatically assigned to another worker. However, this code does not transfer the state between workers, leaving the state inconsistent. - state = new State(); - } - - } else { - log.info("Initializing with new state"); - state = new State(); - } - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); EventStreamReader reader = clientFactory.createReader( readerId, @@ -102,7 +76,7 @@ public Void call() { Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); Path transactionIdFilePath = checkpointDirPath.resolve(ExactlyOnceMultithreadedProcessor.CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + this.workerIndex); - // Must ensure that txnId is persisted to latest state before committing transaction! + // Must ensure that txnId is persisted before committing transaction! // Do not commit transaction here. Instead write TxnId to checkpoint directory. Master will read all TxnIds and commit transactions. String transactionIds = ""; @@ -112,17 +86,6 @@ public Void call() { transaction = null; } Files.write(transactionIdFilePath, transactionIds.getBytes(StandardCharsets.UTF_8)); - - // Write state to checkpoint directory - Path statePath = checkpointDirPath.resolve(STATE_FILE_NAME_PREFIX + this.workerIndex); - log.info("statePath={}", statePath.toString()); - try (FileOutputStream fos = new FileOutputStream(statePath.toString()); - ObjectOutputStream oos = new ObjectOutputStream(fos)) { - oos.writeObject(state); - oos.flush(); - fos.getFD().sync(); - } - } else if (eventRead.getEvent() != null) { eventCounter++; log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); @@ -139,12 +102,9 @@ public Void call() { long generatedSum = Long.parseLong(cols[3]); String generatedTimestampStr = cols[4]; - // Process the input event and update the state. - state.sum += intData; + // Process the input event. String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - if (eventCounter == 10) return null; - // Build the output event. String message = String.join(",", String.format("%06d", generatedEventCounter), @@ -152,7 +112,6 @@ public Void call() { routingKey, String.format("%02d", intData), String.format("%08d", generatedSum), - String.format("%08d", state.sum), String.format("%03d", workerIndex), generatedTimestampStr, processedTimestampStr, diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java index c67cc729..b9f0aa9a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -18,6 +18,9 @@ import org.slf4j.LoggerFactory; import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; import java.text.SimpleDateFormat; import java.util.Date; import java.util.UUID; @@ -28,7 +31,6 @@ * and writing each output event to another Pravega stream. * * This runs only a single thread. - * Upon restart, it reprocesses the entire input stream and recreates the output stream. * * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} * to view the output events. @@ -40,12 +42,10 @@ public class NonRecoverableSingleThreadedProcessor { private static final int READER_TIMEOUT_MS = 2000; - public final String scope; - public final String inputStreamName; - public final String outputStreamName; - public final URI controllerURI; - - public State state; + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; public NonRecoverableSingleThreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { this.scope = scope; @@ -96,51 +96,47 @@ public void run() throws Exception { new UTF8StringSerializer(), EventWriterConfig.builder().build())) { - // Initialize state. - state = new State(); + long eventCounter = 0; - EventRead event; - for (int i = 0; ; i++) { + for (; ; ) { // Read input event. - try { - event = reader.readNextEvent(READER_TIMEOUT_MS); - } catch (ReinitializationRequiredException e) { - // There are certain circumstances where the reader needs to be reinitialized - log.error("Read error", e); - throw e; - } + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + log.debug("readEvents: eventRead={}", eventRead); - if (event.getEvent() != null) { - log.info("Read event '{}'", event.getEvent()); + if (eventRead.getEvent() != null) { + eventCounter++; + log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); // Parse input event. - String[] cols = event.getEvent().split(","); - String routingKey = cols[0]; - long intData = Long.parseLong(cols[1]); - long generatedIndex = Long.parseLong(cols[2]); - String generatedTimestampStr = cols[3]; - - // Process the input event and update the state. - state.sum += intData; + String[] cols = eventRead.getEvent().split(","); + long generatedEventCounter = Long.parseLong(cols[0]); + String routingKey = cols[1]; + long intData = Long.parseLong(cols[2]); + long generatedSum = Long.parseLong(cols[3]); + String generatedTimestampStr = cols[4]; + + // Process the input event. String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); // Build the output event. String message = String.join(",", + String.format("%06d", generatedEventCounter), + String.format("%06d", eventCounter), routingKey, String.format("%02d", intData), - String.format("%08d", generatedIndex), - String.format("%08d", i), + String.format("%08d", generatedSum), + String.format("%03d", 0), generatedTimestampStr, processedTimestampStr, - String.format("%d", state.sum)); + ""); // Write the output event. - log.info("Writing message '{}' with routing key '{}' to stream {}/{}", - message, routingKey, scope, outputStreamName); + log.info("eventCounter={}, event={}", + String.format("%06d", eventCounter), + message); final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); } } } } - } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java index 8913b21a..8adcf369 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java @@ -49,7 +49,7 @@ public static long getCheckpointPeriodMs() { } public static long getCheckpointTimeoutMs() { - return Long.parseLong(getEnvVar("CHECKPOINT_TIMEOUT_MS", "10000")); + return Long.parseLong(getEnvVar("CHECKPOINT_TIMEOUT_MS", "120000")); } public static long getTransactionTimeoutMs() { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index 4143f8ca..a90a9a3d 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -6,14 +6,10 @@ The examples in this directory are intended to illustrate how exactly-once semantics can be achieved with Pravega. To make it clear to the reader, the entirety of the stream processor is defined -in a single source file of under 500 lines. -In particular, these illustrative examples do not use Apache Flink. - -As one reads and understands this example exactly-once processor, they should -understand that enterprise-grade stream processing at scale requires -a sophisticated stream processing system like Apache Flink. -In addition to providing exactly-once processing, it also handles -windowing, aggregations, event time processing, state management, and a lot more. +within this package with minimal dependencies. +In particular, these illustrative examples do *not* use Apache Flink. +In addition to exactly-once semantics, Apache Flink provides +windowing, aggregations, event time processing, and stateful processing. These examples include: @@ -22,7 +18,7 @@ These examples include: and writes them to a Pravega stream (referred to as stream1). - [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java): - This application continuously reads events from stream1, performs a stateful computation + This application continuously reads events from stream1, performs a stateless computation to generate output events, and writes the output event to another Pravega stream (referred to as stream2). It uses the exactly-once algorithms described below. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java deleted file mode 100644 index b3836b81..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/State.java +++ /dev/null @@ -1,20 +0,0 @@ -package io.pravega.example.streamprocessing; - -import java.io.Serializable; - -class State implements Serializable { - private static final long serialVersionUID = -275148988691911596L; - - long sum; - - public State() { - this.sum = 0; - } - - @Override - public String toString() { - return "State{" + - "sum=" + sum + - '}'; - } -} From 178957e459cf541907ad7d7eae46dd3654f93d17 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 1 May 2019 15:43:48 -0700 Subject: [PATCH 05/78] Issue 191: Cleanup - Moved SimpleDataFormat creation outside of loop. - Added missing license headers. - Updated various comments. - Removed unneeded variable. - Updated routing key generation. Signed-off-by: Claudio Fahey --- .../example/streamprocessing/EventGenerator.java | 5 +++-- .../ExactlyOnceMultithreadedProcessor.java | 15 +++++++++++++-- .../ExactlyOnceMultithreadedProcessorWorker.java | 11 ++++++++++- .../NonRecoverableSingleThreadedProcessor.java | 2 ++ .../example/streamprocessing/Parameters.java | 10 ++++++++++ 5 files changed, 38 insertions(+), 5 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 074c90d8..c2c248b1 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -72,12 +72,13 @@ public void run() throws Exception { EventWriterConfig.builder().build())) { long eventCounter = 0; long sum = 0; + final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); for (;;) { eventCounter++; - String routingKey = String.format("rk%02d", eventCounter % 10); long intData = rand.nextInt(100); + String routingKey = Long.toString(intData); sum += intData; - String generatedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + String generatedTimestampStr = dateFormat.format(new Date()); String message = String.join(",", String.format("%06d", eventCounter), routingKey, diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java index 9786c55a..4d37c3b8 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import io.pravega.client.ClientFactory; @@ -111,7 +121,8 @@ public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, S new UTF8StringSerializer(), EventWriterConfig.builder().build()); - // Create a reader group manager. It must remain open to allow manual checkpoints to work. + // Create a reader group manager. + // It must remain open so that readerGroup.initiateCheckpoint returns. readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); final ReaderGroupConfig readerGroupConfig = builder.build(); @@ -125,7 +136,7 @@ public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, S } /** - * Commit all transactions that are part of a checkpoint. + * Commit all transactions that have been opened since the last checkpoint. * * @param checkpointName */ diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java index 0654f26b..5b5f61b0 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import io.pravega.client.ClientFactory; @@ -19,7 +29,6 @@ public class ExactlyOnceMultithreadedProcessorWorker implements Callable { private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessorWorker.class); - private static final String STATE_FILE_NAME_PREFIX = "state-worker-"; private static final int READER_TIMEOUT_MS = 2000; private final int workerIndex; diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java index b9f0aa9a..594d6ebb 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -134,6 +134,8 @@ public void run() throws Exception { log.info("eventCounter={}, event={}", String.format("%06d", eventCounter), message); + // Note that writeEvent returns a future. When the event has been durably persisted + // to Pravega, the future will complete. final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java index 8adcf369..fc951574 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import java.net.URI; From 90af788abc5d2e1ac8522873d9a82a3cb1483062 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 1 May 2019 17:42:31 -0700 Subject: [PATCH 06/78] Issue 191: Allow variable number of workers; simplify recovery - Now allows the number of workers to be different from the number of workers in previous executions. - Simplified the recovery process by terminating the entire process upon any exceptions during checkpointing. - Changed formatting of routing key. Signed-off-by: Claudio Fahey --- .../streamprocessing/EventGenerator.java | 2 +- .../ExactlyOnceMultithreadedProcessor.java | 50 ++++++++++++------- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index c2c248b1..72c4fa7d 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -76,7 +76,7 @@ public void run() throws Exception { for (;;) { eventCounter++; long intData = rand.nextInt(100); - String routingKey = Long.toString(intData); + String routingKey = String.format("%02d", intData); sum += intData; String generatedTimestampStr = dateFormat.format(new Date()); String message = String.join(",", diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java index 4d37c3b8..828d59ba 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java @@ -17,6 +17,7 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.net.URI; @@ -136,7 +137,7 @@ public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, S } /** - * Commit all transactions that have been opened since the last checkpoint. + * Commit all transactions that are part of a checkpoint. * * @param checkpointName */ @@ -146,21 +147,23 @@ private void commitTransactions(String checkpointName) { // Read the contents of all pravega-transactions-worker-XX files. // These files contain the Pravega transaction IDs that must be committed now. Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - List txnIds = IntStream - .range(0, numWorkers) - .parallel() - .boxed() - .map(workerIndex -> checkpointDirPath.resolve(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + workerIndex)) - .filter(Files::exists) - .flatMap(path -> { - try { - return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); - } catch (IOException e) { - throw new RuntimeException(e); - } - }) - .map(UUID::fromString) - .collect(Collectors.toList()); + List txnIds = null; + try { + txnIds = Files.list(checkpointDirPath) + .filter(path -> path.getFileName().toString().startsWith(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX)) + .parallel() + .flatMap(path -> { + try { + return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .map(UUID::fromString) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new RuntimeException(e); + } log.info("commitTransactions: txnIds={}", txnIds); @@ -214,7 +217,8 @@ public Void call() throws InterruptedException { } /** - * Initiate a checkpoint, wait for it to complete, and write the checkpoint to the state. + * Initiate a checkpoint, wait for it to complete, write the checkpoint to the state, + * and commit transactions. */ private void performCheckpoint() { final String checkpointName = UUID.randomUUID().toString(); @@ -254,8 +258,9 @@ private void performCheckpoint() { cleanCheckpointDirectory(checkpointDirPath); } catch (final Exception e) { - log.warn("performCheckpoint: Error performing checkpoint", e); - // Ignore error. We will retry when we are scheduled again. + // If any exception occurs, this application will abnormally terminate. + // Upon restart, it will resume from the last successful checkpoint. + panic(e); } log.info("performCheckpoint: END: checkpointName={}", checkpointName); } @@ -295,10 +300,17 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO } }); } catch (IOException e) { + // Any errors here are non-fatal. The next call to this function + // will attempt to clean anything that was missed. log.warn("cleanCheckpointDirectory", e); } } + private void panic(Exception e) { + log.error("Aborting due to fatal exception", e); + System.exit(1); + } + public static void main(String[] args) throws Exception { ExactlyOnceMultithreadedProcessor master = new ExactlyOnceMultithreadedProcessor( Parameters.getScope(), From c2c5fe6caf2ef16b64907c4e26bf69c46952f9b5 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 31 May 2020 22:09:31 +0000 Subject: [PATCH 07/78] Update to use EventStreamClientFactory. Removed ExactlyOnceMultithreadedProcessor. Signed-off-by: Claudio Fahey --- .../streamprocessing/EventDebugSink.java | 14 +- .../streamprocessing/EventGenerator.java | 9 +- .../ExactlyOnceMultithreadedProcessor.java | 323 ------------------ ...actlyOnceMultithreadedProcessorWorker.java | 146 -------- ...NonRecoverableSingleThreadedProcessor.java | 19 +- 5 files changed, 29 insertions(+), 482 deletions(-) delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 174d38d1..91da8fd7 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -10,10 +10,17 @@ */ package io.pravega.example.streamprocessing; -import io.pravega.client.ClientFactory; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.*; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; @@ -49,6 +56,7 @@ public static void main(String[] args) throws Exception { } public void run() throws Exception { + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); try (StreamManager streamManager = StreamManager.create(controllerURI)) { streamManager.createScope(scope); StreamConfiguration streamConfig = StreamConfiguration.builder() @@ -69,7 +77,7 @@ public void run() throws Exception { readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); } - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); EventStreamReader reader = clientFactory.createReader( "reader", readerGroup, diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 72c4fa7d..0e8dd6f4 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -10,7 +10,8 @@ */ package io.pravega.example.streamprocessing; -import io.pravega.client.ClientFactory; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventStreamWriter; import io.pravega.client.stream.EventWriterConfig; @@ -52,6 +53,7 @@ public static void main(String[] args) throws Exception { } public void run() throws Exception { + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); try (StreamManager streamManager = StreamManager.create(controllerURI)) { streamManager.createScope(scope); StreamConfiguration streamConfig = StreamConfiguration.builder() @@ -64,8 +66,7 @@ public void run() throws Exception { } Random rand = new Random(42); - - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); EventStreamWriter writer = clientFactory.createEventWriter( outputStreamName, new UTF8StringSerializer(), @@ -90,7 +91,7 @@ public void run() throws Exception { String.format("%06d", eventCounter), String.format("%08d", sum), message); - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); Thread.sleep(1000); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java deleted file mode 100644 index 828d59ba..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessor.java +++ /dev/null @@ -1,323 +0,0 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ -package io.pravega.example.streamprocessing; - -import io.pravega.client.ClientFactory; -import io.pravega.client.admin.ReaderGroupManager; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.*; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.net.URI; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.nio.file.*; -import java.nio.file.attribute.BasicFileAttributes; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.*; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -/** - * A simple example that demonstrates reading events from a Pravega stream, processing each event, - * and writing each output event to another Pravega stream. - * - * This supports multiple worker threads. - * Upon restart, it restarts from the last successful checkpoint and guarantees exactly-once semantics. - * Workers must be stateless. - * - * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} - * to view the output events. - * - * See also {@link ExactlyOnceMultithreadedProcessorWorker}. - */ - -public class ExactlyOnceMultithreadedProcessor implements Callable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessor.class); - - private static final String PRAVEGA_CHECKPOINT_FILE_NAME = "pravega-checkpoint"; - private static final String LATEST_CHECKPOINT_NAME_FILE_NAME = "latest"; - static final String CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX = "pravega-transactions-worker-"; - - private final String scope; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - private final int numWorkers; - private final String readerGroupName; - private final ReaderGroup readerGroup; - private final ScheduledExecutorService initiateCheckpointExecutor; - private final ScheduledExecutorService performCheckpointExecutor; - private final ExecutorService workerExecutor; - private final ReaderGroupManager readerGroupManager; - private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - private final Path latestCheckpointNamePath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME); - private final long checkpointPeriodMs = Parameters.getCheckpointPeriodMs(); - private final long checkpointTimeoutMs = Parameters.getCheckpointTimeoutMs(); - private final boolean startFromCheckpoint; - private final String startFromCheckpointName; - private final ClientFactory clientFactory; - private final EventStreamWriter writer; - - public ExactlyOnceMultithreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI, int numWorkers) throws Exception { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - this.numWorkers = numWorkers; - - ReaderGroupConfig.ReaderGroupConfigBuilder builder = ReaderGroupConfig.builder() - .disableAutomaticCheckpoints(); - - // Load the last checkpoint. - startFromCheckpoint = latestCheckpointNamePath.toFile().exists(); - if (startFromCheckpoint) { - // Read the name of the checkpoint from the file /tmp/checkpoint/latest. - String checkpointName = StandardCharsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(latestCheckpointNamePath))).toString(); - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Reading Pravega checkpoint from {}", checkpointPath); - Checkpoint checkpoint = Checkpoint.fromBytes(ByteBuffer.wrap(Files.readAllBytes(checkpointPath))); - log.info("Starting from checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - builder = builder.startFromCheckpoint(checkpoint); - startFromCheckpointName = checkpointName; - } else { - log.warn("Checkpoint file {} not found. Starting processing from the earliest event.", checkpointRootPath); - - // Create streams. - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - streamManager.createStream(scope, outputStreamName, streamConfig); - } - - // Create a reader group that starts from the earliest event. - builder = builder.stream(Stream.of(scope, inputStreamName)); - startFromCheckpointName = null; - } - - clientFactory = ClientFactory.withScope(scope, controllerURI); - writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build()); - - // Create a reader group manager. - // It must remain open so that readerGroup.initiateCheckpoint returns. - readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI); - - final ReaderGroupConfig readerGroupConfig = builder.build(); - readerGroupName = UUID.randomUUID().toString().replace("-", ""); - readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); - readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - - initiateCheckpointExecutor = Executors.newScheduledThreadPool(1); - performCheckpointExecutor = Executors.newScheduledThreadPool(1); - workerExecutor = Executors.newFixedThreadPool(numWorkers); - } - - /** - * Commit all transactions that are part of a checkpoint. - * - * @param checkpointName - */ - private void commitTransactions(String checkpointName) { - log.info("commitTransactions: BEGIN"); - - // Read the contents of all pravega-transactions-worker-XX files. - // These files contain the Pravega transaction IDs that must be committed now. - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - List txnIds = null; - try { - txnIds = Files.list(checkpointDirPath) - .filter(path -> path.getFileName().toString().startsWith(CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX)) - .parallel() - .flatMap(path -> { - try { - return Files.readAllLines(path, StandardCharsets.UTF_8).stream(); - } catch (IOException e) { - throw new RuntimeException(e); - } - }) - .map(UUID::fromString) - .collect(Collectors.toList()); - } catch (IOException e) { - throw new RuntimeException(e); - } - - log.info("commitTransactions: txnIds={}", txnIds); - - // Commit all transactions that have been opened and flushed since the last checkpoint. - txnIds.parallelStream().forEach(txnId -> { - try { - Transaction transaction = writer.getTxn(txnId); - Transaction.Status status = transaction.checkStatus(); - log.info("commitTransaction: transaction {} status is {}", transaction.getTxnId(), status); - if (status == Transaction.Status.OPEN) { - log.info("commitTransaction: committing {}", transaction.getTxnId()); - transaction.commit(); - log.info("commitTransaction: committed {}", transaction.getTxnId()); - } - } catch (TxnFailedException e) { - throw new RuntimeException(e); - } - }); - - log.info("commitTransactions: END"); - } - - public Void call() throws InterruptedException { - // It is possible that the checkpoint was completely written but that some or all Pravega transactions - // have not been committed. This will ensure that they are. - if (startFromCheckpoint) { - commitTransactions(startFromCheckpointName); - } - - // Schedule periodic task to initiate checkpoints. - // If any execution of this task takes longer than its period, then subsequent executions may start late, but will not concurrently execute. - initiateCheckpointExecutor.scheduleAtFixedRate(this::performCheckpoint, checkpointPeriodMs, checkpointPeriodMs, TimeUnit.MILLISECONDS); - - // Start workers and wait for all of them to terminate. - workerExecutor.invokeAll( - IntStream.range(0, numWorkers).boxed().map( - workerIndex -> new ExactlyOnceMultithreadedProcessorWorker( - workerIndex, scope, readerGroupName, startFromCheckpointName, outputStreamName, controllerURI) - ).collect(Collectors.toList())); - - log.info("call: shutting down"); - workerExecutor.shutdownNow(); - performCheckpointExecutor.shutdownNow(); - initiateCheckpointExecutor.shutdownNow(); - readerGroup.close(); - readerGroupManager.close(); - writer.close(); - clientFactory.close(); - log.info("call: END"); - return null; - } - - /** - * Initiate a checkpoint, wait for it to complete, write the checkpoint to the state, - * and commit transactions. - */ - private void performCheckpoint() { - final String checkpointName = UUID.randomUUID().toString(); - log.info("performCheckpoint: BEGIN: checkpointName={}", checkpointName); - try { - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - checkpointDirPath.toFile().mkdirs(); - - final Set onlineReaders = readerGroup.getOnlineReaders(); - log.info("performCheckpoint: onlineReaders ({})={}", onlineReaders.size(), onlineReaders); - log.info("performCheckpoint: Calling initiateCheckpoint; checkpointName={}", checkpointName); - CompletableFuture checkpointFuture = readerGroup.initiateCheckpoint(checkpointName, performCheckpointExecutor); - Checkpoint checkpoint = checkpointFuture.get(checkpointTimeoutMs, TimeUnit.MILLISECONDS); - // At this point, all workers have received and processed the checkpoint. - log.info("performCheckpoint: Checkpoint completed; checkpointName={}, positions={}", checkpointName, checkpoint.asImpl().getPositions()); - - Path checkpointPath = checkpointDirPath.resolve(PRAVEGA_CHECKPOINT_FILE_NAME); - log.info("Writing Pravega checkpoint to {}", checkpointPath); - try (FileOutputStream fos = new FileOutputStream(checkpointPath.toFile())) { - fos.write(checkpoint.toBytes().array()); - fos.flush(); - fos.getFD().sync(); - } - - // Create "latest" file that indicates the latest checkpoint name. - // This file must be updated atomically. - Path latestTmpCheckpointPath = checkpointRootPath.resolve(LATEST_CHECKPOINT_NAME_FILE_NAME + ".tmp"); - try (FileOutputStream fos = new FileOutputStream(latestTmpCheckpointPath.toFile())) { - fos.write(checkpointName.getBytes(StandardCharsets.UTF_8)); - fos.flush(); - fos.getFD().sync(); - } - Files.move(latestTmpCheckpointPath, latestCheckpointNamePath, StandardCopyOption.ATOMIC_MOVE); - - // Read list of TxnIds from checkpoint directory written by all workers and commit all transactions. - commitTransactions(checkpointName); - - cleanCheckpointDirectory(checkpointDirPath); - } catch (final Exception e) { - // If any exception occurs, this application will abnormally terminate. - // Upon restart, it will resume from the last successful checkpoint. - panic(e); - } - log.info("performCheckpoint: END: checkpointName={}", checkpointName); - } - - /** - * Delete everything in the checkpoint root path (/tmp/checkpoint) except the "latest" file and - * the latest checkpoint directory. - * - * @param keepCheckpointDirPath The latest checkpoint directory which will not be deleted. - */ - private void cleanCheckpointDirectory(Path keepCheckpointDirPath) { - try { - Files.walkFileTree(checkpointRootPath, - new SimpleFileVisitor() { - @Override - public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes basicFileAttributes) throws IOException { - if (dir.equals(keepCheckpointDirPath)) { - return FileVisitResult.SKIP_SUBTREE; - } - return FileVisitResult.CONTINUE; - } - - @Override - public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { - if (!dir.equals(checkpointRootPath)) { - Files.delete(dir); - } - return FileVisitResult.CONTINUE; - } - - @Override - public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { - if (!file.equals(latestCheckpointNamePath)) { - Files.delete(file); - } - return FileVisitResult.CONTINUE; - } - }); - } catch (IOException e) { - // Any errors here are non-fatal. The next call to this function - // will attempt to clean anything that was missed. - log.warn("cleanCheckpointDirectory", e); - } - } - - private void panic(Exception e) { - log.error("Aborting due to fatal exception", e); - System.exit(1); - } - - public static void main(String[] args) throws Exception { - ExactlyOnceMultithreadedProcessor master = new ExactlyOnceMultithreadedProcessor( - Parameters.getScope(), - Parameters.getStream1Name(), - Parameters.getStream2Name(), - Parameters.getControllerURI(), - Parameters.getNumWorkers()); - master.call(); - } -} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java deleted file mode 100644 index 5b5f61b0..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ExactlyOnceMultithreadedProcessorWorker.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ -package io.pravega.example.streamprocessing; - -import io.pravega.client.ClientFactory; -import io.pravega.client.stream.*; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import org.slf4j.LoggerFactory; - -import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.concurrent.Callable; - -/** - * See {@link ExactlyOnceMultithreadedProcessor}. - */ -public class ExactlyOnceMultithreadedProcessorWorker implements Callable { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(ExactlyOnceMultithreadedProcessorWorker.class); - - private static final int READER_TIMEOUT_MS = 2000; - - private final int workerIndex; - private final String scope; - private final String readerGroupName; - private final boolean startFromCheckpoint; - private final String startFromCheckpointName; - private final String outputStreamName; - private final URI controllerURI; - private final String readerId; - private final Path checkpointRootPath = Parameters.getCheckpointRootPath(); - - public ExactlyOnceMultithreadedProcessorWorker(int workerIndex, String scope, String readerGroupName, String startFromCheckpointName, String outputStreamName, URI controllerURI) { - this.workerIndex = workerIndex; - this.scope = scope; - this.readerGroupName = readerGroupName; - this.startFromCheckpointName = startFromCheckpointName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - readerId = "worker-" + this.workerIndex; - startFromCheckpoint = startFromCheckpointName != null; - } - - public Void call() { - Thread.currentThread().setName("worker-" + workerIndex); - log.info("BEGIN"); - - try { - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); - EventStreamReader reader = clientFactory.createReader( - readerId, - readerGroupName, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder() - .transactionTimeoutTime(Parameters.getTransactionTimeoutMs()) - .build())) { - - Transaction transaction = null; - long eventCounter = 0; - - for (; ; ) { - // Read input event. - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - log.debug("readEvents: eventRead={}", eventRead); - - if (eventRead.isCheckpoint()) { - // Note that next call readNextEvent will indicate to Pravega that we are done with the checkpoint. - String checkpointName = eventRead.getCheckpointName(); - log.info("Got checkpoint {}", eventRead.getCheckpointName()); - Path checkpointDirPath = checkpointRootPath.resolve(checkpointName); - Path transactionIdFilePath = checkpointDirPath.resolve(ExactlyOnceMultithreadedProcessor.CHECKPOINT_TRANSACTION_ID_FILE_NAME_PREFIX + this.workerIndex); - - // Must ensure that txnId is persisted before committing transaction! - // Do not commit transaction here. Instead write TxnId to checkpoint directory. Master will read all TxnIds and commit transactions. - - String transactionIds = ""; - if (transaction != null) { - transaction.flush(); - transactionIds = transaction.getTxnId().toString(); - transaction = null; - } - Files.write(transactionIdFilePath, transactionIds.getBytes(StandardCharsets.UTF_8)); - } else if (eventRead.getEvent() != null) { - eventCounter++; - log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); - - if (transaction == null) { - transaction = writer.beginTxn(); - } - - // Parse input event. - String[] cols = eventRead.getEvent().split(","); - long generatedEventCounter = Long.parseLong(cols[0]); - String routingKey = cols[1]; - long intData = Long.parseLong(cols[2]); - long generatedSum = Long.parseLong(cols[3]); - String generatedTimestampStr = cols[4]; - - // Process the input event. - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - String.format("%06d", generatedEventCounter), - String.format("%06d", eventCounter), - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedSum), - String.format("%03d", workerIndex), - generatedTimestampStr, - processedTimestampStr, - transaction.getTxnId().toString()); - - // Write the output event. - log.info("eventCounter={}, event={}", - String.format("%06d", eventCounter), - message); - transaction.writeEvent(routingKey, message); - } - } - } - } catch (Exception e) { - log.error("Fatal Error", e); - // We don't handle incremental recovery of a single failed worker. - // Stop the entire process (master and all workers). - // When it is restarted, recovery of all workers will begin. - System.exit(1); - } - return null; - } -} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java index 594d6ebb..3fe6708f 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java @@ -10,17 +10,23 @@ */ package io.pravega.example.streamprocessing; -import io.pravega.client.ClientFactory; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.*; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; import java.text.SimpleDateFormat; import java.util.Date; import java.util.UUID; @@ -64,6 +70,7 @@ public static void main(String[] args) throws Exception { } public void run() throws Exception { + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); try (StreamManager streamManager = StreamManager.create(controllerURI)) { streamManager.createScope(scope); StreamConfiguration streamConfig = StreamConfiguration.builder() @@ -85,7 +92,7 @@ public void run() throws Exception { readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); } - try (ClientFactory clientFactory = ClientFactory.withScope(scope, controllerURI); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); EventStreamReader reader = clientFactory.createReader( "reader", readerGroup, From 666ea50dc81f38d949bb899ee275ca9014f5fd74 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Mon, 1 Jun 2020 00:10:35 +0000 Subject: [PATCH 08/78] Add test structure Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 4 + .../src/main/resources/logback.xml | 2 +- .../pravega/example/StreamProcessingTest.java | 35 ++ .../java/io/pravega/utils/SetupUtils.java | 361 ++++++++++++++++++ 4 files changed, 401 insertions(+), 1 deletion(-) create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 8b822882..6fafd4c3 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -20,12 +20,16 @@ archivesBaseName = 'pravega-client-examples' resources ext { +// junitJupiterVersion = '5.3.1' junitVersion = '4.11' commonsCLIVersion = '1.4' } dependencies { testCompile "junit:junit:${junitVersion}" +// testCompile "org.junit.jupiter:junit-jupiter-engine:${junitJupiterVersion}" + testCompile "io.pravega:pravega-standalone:${pravegaVersion}" + compileOnly "org.projectlombok:lombok:${lombokVersion}" compile "io.pravega:pravega-client:${pravegaVersion}", "io.pravega:pravega-common:${pravegaVersion}", diff --git a/pravega-client-examples/src/main/resources/logback.xml b/pravega-client-examples/src/main/resources/logback.xml index ae1a2a69..34f9c7c0 100644 --- a/pravega-client-examples/src/main/resources/logback.xml +++ b/pravega-client-examples/src/main/resources/logback.xml @@ -18,7 +18,7 @@ - + diff --git a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java new file mode 100644 index 00000000..bf7a8a36 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java @@ -0,0 +1,35 @@ +package io.pravega.example; + +import io.pravega.utils.SetupUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +public class StreamProcessingTest { + static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); + + protected static final AtomicReference SETUP_UTILS = new AtomicReference<>(); + + @BeforeClass + public static void setup() throws Exception { + SETUP_UTILS.set(new SetupUtils()); + if( SETUP_UTILS.get() == null) throw new RuntimeException("This is null"); + SETUP_UTILS.get().startAllServices(); + } + + @AfterClass + public static void tearDown() throws Exception { + SETUP_UTILS.get().stopAllServices(); + } + + @Test + public void basicWriterTest() throws IOException { + String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); + log.info("Test case: {}", methodName); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java new file mode 100644 index 00000000..dc92bff6 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java @@ -0,0 +1,361 @@ +/* + * Copyright 2019 Flavio Junqueira + * + * Licensed 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 io.pravega.utils; + +import com.google.common.base.Preconditions; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.impl.Controller; +import io.pravega.client.stream.impl.ControllerImpl; +import io.pravega.client.stream.impl.ControllerImplConfig; +import io.pravega.common.concurrent.ExecutorServiceHelpers; +import io.pravega.local.InProcPravegaCluster; +import lombok.Cleanup; +import lombok.Getter; +import lombok.Setter; +import org.apache.commons.lang3.RandomStringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; +import java.io.IOException; +import java.io.InputStream; +import java.net.ServerSocket; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Utility functions for creating the test setup. + */ +//@Slf4j +@NotThreadSafe +public final class SetupUtils { + + static Logger log = LoggerFactory.getLogger(SetupUtils.class); + private static final ScheduledExecutorService DEFAULT_SCHEDULED_EXECUTOR_SERVICE = ExecutorServiceHelpers.newScheduledThreadPool(3, "SetupUtils"); + + private final PravegaGateway gateway; + + // Manage the state of the class. + private final AtomicBoolean started = new AtomicBoolean(false); + + // auth enabled by default. Set it to false to disable Pravega authentication and authorization. + @Setter + private boolean enableAuth = false; + + // Set to true to enable TLS + @Setter + private boolean enableTls = false; + + @Setter + private boolean enableHostNameValidation = false; + + private boolean enableRestServer = true; + + // The test Scope name. + @Getter + private final String scope = RandomStringUtils.randomAlphabetic(20); + + public SetupUtils() { + this(System.getProperty("pravega.uri")); + } + + public SetupUtils(String externalUri) { + log.info("SetupUtils constructor"); + if (externalUri != null) { + log.info("Using Pravega services at {}.", externalUri); + gateway = new ExternalPravegaGateway(URI.create(externalUri)); + } else { + log.info("Starting in-process Pravega services."); + gateway = new InProcPravegaGateway(); + } + log.info("Done with constructor {}", gateway.toString()); + } + + + /** + * Start all pravega related services required for the test deployment. + * + * @throws Exception on any errors. + */ + public void startAllServices() throws Exception { + log.info("Starting all services"); + if (!this.started.compareAndSet(false, true)) { + log.warn("Services already started, not attempting to start again"); + return; + } + log.info("Starting gateway"); + gateway.start(); + } + + /** + * Stop the pravega cluster and release all resources. + * + * @throws Exception on any errors. + */ + public void stopAllServices() throws Exception { + if (!this.started.compareAndSet(true, false)) { + log.warn("Services not yet started or already stopped, not attempting to stop"); + return; + } + + try { + gateway.stop(); + } catch (Exception e) { + log.warn("Services did not stop cleanly (" + e.getMessage() + ")", e); + } + } + + /** + * Get resources as temp file. + * + * @param resourceName Name of the resource. + * + * @return Path of the temp file. + */ + static String getFileFromResource(String resourceName) { + try { + Path tempPath = Files.createTempFile("test-", ".tmp"); + tempPath.toFile().deleteOnExit(); + try (InputStream stream = SetupUtils.class.getClassLoader().getResourceAsStream(resourceName)) { + Files.copy(SetupUtils.class.getClassLoader().getResourceAsStream(resourceName), tempPath, StandardCopyOption.REPLACE_EXISTING); + } + return tempPath.toFile().getAbsolutePath(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + /** + * Fetch the controller endpoint for this cluster. + * + * @return URI The controller endpoint to connect to this cluster. + */ + public URI getControllerUri() { + return getClientConfig().getControllerURI(); + } + + /** + * Fetch the client configuration with which to connect to the controller. + */ + public ClientConfig getClientConfig() { + return this.gateway.getClientConfig(); + } + + /** + * Create a controller facade for this cluster. + * @return The controller facade, which must be closed by the caller. + */ + public Controller newController() { + ControllerImplConfig config = ControllerImplConfig.builder() + .clientConfig(getClientConfig()) + .build(); + return new ControllerImpl(config, DEFAULT_SCHEDULED_EXECUTOR_SERVICE); + } + + /** + * Create a {@link EventStreamClientFactory} for this cluster and scope. + */ + public EventStreamClientFactory newClientFactory() { + return EventStreamClientFactory.withScope(this.scope, getClientConfig()); + } + + /** + * Create the test stream. + * + * @param streamName Name of the test stream. + * @param numSegments Number of segments to be created for this stream. + * + * @throws Exception on any errors. + */ + public void createTestStream(final String streamName, final int numSegments) + throws Exception { + Preconditions.checkState(this.started.get(), "Services not yet started"); + Preconditions.checkNotNull(streamName); + Preconditions.checkArgument(numSegments > 0); + + @Cleanup + StreamManager streamManager = StreamManager.create(getClientConfig()); + streamManager.createScope(this.scope); + streamManager.createStream(this.scope, streamName, + StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.fixed(numSegments)) + .build()); + log.info("Created stream: " + streamName); + } + + /** + * Get the stream. + * + * @param streamName Name of the test stream. + * + * @return a Stream + */ + public Stream getStream(final String streamName) { + return Stream.of(this.scope, streamName); + } + + /** + * A gateway interface to Pravega for integration test purposes. + */ + private interface PravegaGateway { + /** + * Starts the gateway. + */ + void start() throws Exception; + + /** + * Stops the gateway. + */ + void stop() throws Exception; + + /** + * Gets the client configuration with which to connect to the controller. + */ + ClientConfig getClientConfig(); + } + + class InProcPravegaGateway implements PravegaGateway { + + // The pravega cluster. + private InProcPravegaCluster inProcPravegaCluster = null; + + @Override + public void start() throws Exception { + log.info("Starting gateway"); + int zkPort = PortUtils.getAvailableListenPort(); + int controllerPort = PortUtils.getAvailableListenPort(); + int hostPort = PortUtils.getAvailableListenPort(); + int restPort = PortUtils.getAvailableListenPort(); + + log.info("Building"); + this.inProcPravegaCluster = InProcPravegaCluster.builder() + .isInProcZK(true) + .secureZK(enableTls) //configure ZK for security + .zkUrl("localhost:" + zkPort) + .zkPort(zkPort) + .isInMemStorage(true) + .isInProcController(true) + .controllerCount(1) + .restServerPort(restPort) + .enableRestServer(enableRestServer) + .isInProcSegmentStore(true) + .segmentStoreCount(1) + .containerCount(4) + .enableMetrics(false) + .enableAuth(enableAuth) + .enableTls(enableTls) + //.certFile(getFileFromResource(CERT_FILE)) // pravega #2519 + //.keyFile(getFileFromResource(KEY_FILE)) + //.jksKeyFile(getFileFromResource(STANDALONE_KEYSTORE_FILE)) + //.jksTrustFile(getFileFromResource(STANDALONE_TRUSTSTORE_FILE)) + //.keyPasswordFile(getFileFromResource(STANDALONE_KEYSTORE_PASSWD_FILE)) + //.passwdFile(getFileFromResource(PASSWD_FILE)) + //.userName(PRAVEGA_USERNAME) + //.passwd(PRAVEGA_PASSWORD) + .build(); + log.info("Done building"); + this.inProcPravegaCluster.setControllerPorts(new int[]{controllerPort}); + this.inProcPravegaCluster.setSegmentStorePorts(new int[]{hostPort}); + this.inProcPravegaCluster.start(); + log.info("Initialized Pravega Cluster"); + log.info("Controller port is {}", controllerPort); + log.info("Host port is {}", hostPort); + log.info("REST server port is {}", restPort); + } + + @Override + public void stop() throws Exception { + inProcPravegaCluster.close(); + } + + @Override + public ClientConfig getClientConfig() { + log.info("Getting client config"); + return ClientConfig.builder() + .controllerURI(URI.create(inProcPravegaCluster.getControllerURI())) + //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) + //.validateHostName(enableHostNameValidation) + //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) + .build(); + } + } + + class ExternalPravegaGateway implements PravegaGateway { + + private final URI controllerUri; + + public ExternalPravegaGateway(URI controllerUri) { + this.controllerUri = controllerUri; + } + + @Override + public void start() throws Exception { + } + + @Override + public void stop() throws Exception { + } + + @Override + public ClientConfig getClientConfig() { + return ClientConfig.builder() + .controllerURI(controllerUri) + //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) + //.validateHostName(enableHostNameValidation) + //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) + .build(); + } + } + + static class PortUtils { + // Linux uses ports from range 32768 - 61000. + private static final int BASE_PORT = 32768; + private static final int MAX_PORT_COUNT = 28232; + private static final AtomicInteger NEXT_PORT = new AtomicInteger(1); + + /** + * A helper method to get a random free port. + * + * @return free port. + */ + public static int getAvailableListenPort() { + for (int i = 0; i < MAX_PORT_COUNT; i++) { + int candidatePort = BASE_PORT + NEXT_PORT.getAndIncrement() % MAX_PORT_COUNT; + try { + ServerSocket serverSocket = new ServerSocket(candidatePort); + serverSocket.close(); + return candidatePort; + } catch (IOException e) { + // Do nothing. Try another port. + } + } + throw new IllegalStateException( + String.format("Could not assign port in range %d - %d", BASE_PORT, MAX_PORT_COUNT + BASE_PORT)); + } + } +} From c76d1655e12ebded7da0fcea2df09d297618145c Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Mon, 1 Jun 2020 04:59:04 +0000 Subject: [PATCH 09/78] Add AtLeastOnceProcessor Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 41 +++++++++++ .../pravega/example/StreamProcessingTest.java | 72 +++++++++++++++++-- 2 files changed, 109 insertions(+), 4 deletions(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java new file mode 100644 index 00000000..fd1205c2 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -0,0 +1,41 @@ +package io.pravega.example.streamprocessing; + +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Callable; + +abstract public class AtLeastOnceProcessor implements Callable { + private static final Logger log = LoggerFactory.getLogger(NonRecoverableSingleThreadedProcessor.class); + + private final long readTimeoutMillis; + + public AtLeastOnceProcessor() { + readTimeoutMillis = 1000; + } + + @Override + public Void call() throws Exception { + // TODO: handle case when reader dies (if checkpoint timeout occurs, reset reader group to last successful checkpoint) + try (final EventStreamReader reader = createReader()) { + for (; ; ) { + final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); + log.info("call: eventRead={}", eventRead); + if (eventRead.isCheckpoint()) { + flush(eventRead); + } else if (eventRead.getEvent() != null) { + write(eventRead); + } + } + } + } + + abstract public EventStreamReader createReader(); + + abstract public void write(EventRead eventRead); + + public void flush(EventRead eventRead) { + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java index bf7a8a36..7297f686 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java @@ -1,5 +1,21 @@ package io.pravega.example; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Serializer; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import io.pravega.example.streamprocessing.AtLeastOnceProcessor; import io.pravega.utils.SetupUtils; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -7,7 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; +import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; public class StreamProcessingTest { @@ -18,7 +34,6 @@ public class StreamProcessingTest { @BeforeClass public static void setup() throws Exception { SETUP_UTILS.set(new SetupUtils()); - if( SETUP_UTILS.get() == null) throw new RuntimeException("This is null"); SETUP_UTILS.get().startAllServices(); } @@ -27,9 +42,58 @@ public static void tearDown() throws Exception { SETUP_UTILS.get().stopAllServices(); } + public static void runWorker(final ClientConfig clientConfig, + final String inputStreamName, + final String readerGroup) throws Exception { + + } + @Test - public void basicWriterTest() throws IOException { - String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); + public void basicTest() throws Exception { + final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); + + final String inputStreamName = "stream-" + UUID.randomUUID().toString(); + final String readerGroup = "rg" + UUID.randomUUID().toString().replace("-", ""); + final String scope = SETUP_UTILS.get().getScope(); + final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); + final String readerId = "reader-" + UUID.randomUUID().toString(); + final ReaderConfig readerConfig = ReaderConfig.builder().build(); + final Serializer serializer = new UTF8StringSerializer(); + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, inputStreamName)) + .build(); + final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); + + try (StreamManager streamManager = StreamManager.create(clientConfig)) { + streamManager.createScope(scope); + + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.fixed(3)) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + } + try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { + + inputWriter.writeEvent("CLAUDIO1"); + + AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { + @Override + public EventStreamReader createReader() { + return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); + } + + @Override + public void write(EventRead eventRead) { + } + }; + processor.call(); + // TODO: Wait for correct result and then terminate. + } + } } } From 18b8fb4626d2afa81d4a37d77ca909ceb17c680b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 3 Jun 2020 01:48:03 +0000 Subject: [PATCH 10/78] Add https://github.com/pravega/pravega/blob/c7ac009970787df2633163644691aa3d72fc1a4e/client/src/test/java/io/pravega/client/state/examples/MembershipSynchronizer.java Signed-off-by: Claudio Fahey --- .../MembershipSynchronizer.java | 243 ++++++++++++++++++ 1 file changed, 243 insertions(+) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java new file mode 100644 index 00000000..aeb5e0d2 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -0,0 +1,243 @@ +/** + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + */ +package io.pravega.example.streamprocessing; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.AbstractService; +import io.pravega.client.SynchronizerClientFactory; +import io.pravega.client.state.InitialUpdate; +import io.pravega.client.state.Revision; +import io.pravega.client.state.Revisioned; +import io.pravega.client.state.StateSynchronizer; +import io.pravega.client.state.Update; +import io.pravega.client.stream.impl.JavaSerializer; +import lombok.Data; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +@Slf4j +public class MembershipSynchronizer extends AbstractService { + + /** + * How frequently to update the segment using a heartbeat. + */ + private static final int UPDATE_INTERVAL_MILLIS = 1000; + + /** + * Number of intervals behind before switching to unconditional updates. + */ + private static final int UNCONDITIONAL_UPDATE_THRESHOLD = 3; + + /** + * Number of intervals behind before we should stop executing for safety. + */ + private static final int UNHEALTHY_THRESHOLD = 5; + + /** + * Number of intervals behind before another host should be considered dead. + */ + private static final int DEATH_THRESHOLD = 10; + + private final String instanceId = UUID.randomUUID().toString(); + + private final AtomicBoolean healthy = new AtomicBoolean(); + + private final ScheduledExecutorService executor; + + private final StateSynchronizer stateSync; + private final MembershipListener listener; + private ScheduledFuture task; + + MembershipSynchronizer(String streamName, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, + MembershipListener listener) { + Preconditions.checkNotNull(streamName); + Preconditions.checkNotNull(clientFactory); + Preconditions.checkNotNull(listener); + this.executor = executor; + this.listener = listener; + stateSync = clientFactory.createStateSynchronizer(streamName, + new JavaSerializer(), + new JavaSerializer(), + null); + } + + @Data + private static class LiveInstances + implements Revisioned, Comparable, Serializable, InitialUpdate { + + private static final long serialVersionUID = 1L; + private final String scopedStreamName; + private final Revision revision; + private final Map liveInstances; + private final long vectorTime; + + @Override + public int compareTo(LiveInstances o) { + return revision.compareTo(o.revision); + } + + @Override + public LiveInstances create(String scopedStreamName, Revision revision) { + return new LiveInstances(scopedStreamName, revision, liveInstances, vectorTime); + } + + public boolean isHealthy(String instance) { + long unhealthyThreshold = vectorTime - UNHEALTHY_THRESHOLD * liveInstances.size(); + Long time = liveInstances.get(instance); + return time == null || time >= unhealthyThreshold; + } + + /** + * If a host is behind in it's heartbeating it does not want to become unhealthy as it might + * need to halt execution. So it can use unconditional writes to updates itself more quickly + * by avoiding contention. + */ + public boolean isOverUnconditionalThreshold(String instance) { + long updateThreshold = vectorTime - UNCONDITIONAL_UPDATE_THRESHOLD * liveInstances.size(); + Long time = liveInstances.get(instance); + return time == null || time < updateThreshold; + } + + public List findInstancesThatWillDieBy(long vectorTime) { + long deathThreshold = vectorTime - DEATH_THRESHOLD * liveInstances.size(); + return liveInstances.entrySet() + .stream() + .filter(entry -> entry.getValue() < deathThreshold) + .map(entry -> entry.getKey()) + .collect(Collectors.toList()); + } + + public Set getLiveInstances() { + return liveInstances.keySet(); + } + } + + private class HeartBeater implements Runnable { + @Override + public void run() { + try { + stateSync.fetchUpdates(); + notifyListener(); + if (stateSync.getState().isOverUnconditionalThreshold(instanceId)) { + stateSync.updateState((state, updates) -> { + long vectorTime = state.getVectorTime() + 1; + updates.add(new HeartBeat(instanceId, vectorTime)); + for (String id : state.findInstancesThatWillDieBy(vectorTime)) { + if (!id.equals(instanceId)) { + updates.add(new DeclareDead(id)); + } + } + }); + } else { + stateSync.updateStateUnconditionally(new HeartBeat(instanceId, stateSync.getState().vectorTime)); + stateSync.fetchUpdates(); + } + notifyListener(); + } catch (Exception e) { + log.warn("Encountered an error while heartbeating: " + e); + if (healthy.compareAndSet(true, false)) { + listener.unhealthy(); + } + } + } + } + + private abstract class HeartbeatUpdate implements Update, Serializable { + private static final long serialVersionUID = 1L; + } + + @RequiredArgsConstructor + private final class HeartBeat extends HeartbeatUpdate { + private static final long serialVersionUID = 1L; + private final String name; + private final long timestamp; + + @Override + public LiveInstances applyTo(LiveInstances state, Revision newRevision) { + Map timestamps = new HashMap<>(state.liveInstances); + long vectorTime = Long.max(timestamps.values().stream().max(Long::compare).get(), timestamp); + timestamps.put(name, timestamp); + return new LiveInstances(state.scopedStreamName, + newRevision, + Collections.unmodifiableMap(timestamps), + vectorTime); + } + } + + @RequiredArgsConstructor + private final class DeclareDead extends HeartbeatUpdate { + private static final long serialVersionUID = 1L; + private final String name; + + @Override + public LiveInstances applyTo(LiveInstances state, Revision newRevision) { + Map timestamps = new HashMap<>(state.liveInstances); + timestamps.remove(name); + return new LiveInstances(state.scopedStreamName, + newRevision, + Collections.unmodifiableMap(timestamps), + state.vectorTime); + } + } + + public void notifyListener() { + LiveInstances currentState = stateSync.getState(); + if (currentState.isHealthy(instanceId)) { + if (healthy.compareAndSet(false, true)) { + listener.healthy(); + } + } else { + if (healthy.compareAndSet(true, false)) { + listener.unhealthy(); + } + } + } + + public boolean isCurrentlyHealthy() { + return healthy.get(); + } + + public Set getCurrentMembers() { + return stateSync.getState().getLiveInstances(); + } + + public interface MembershipListener { + void healthy(); + + void unhealthy(); + } + + @Override + protected void doStart() { + task = executor.scheduleAtFixedRate(new HeartBeater(), + UPDATE_INTERVAL_MILLIS, + UPDATE_INTERVAL_MILLIS, + TimeUnit.MILLISECONDS); + } + + @Override + protected void doStop() { + task.cancel(false); + } + +} From 6905085dc398f16507856ef032aa8a7002dde617 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 3 Jun 2020 04:32:12 +0000 Subject: [PATCH 11/78] Add ReaderGroupPruner (incomplete) Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 51 ++++-- .../AtLeastOnceProcessorMain.java | 170 ++++++++++++++++++ .../MembershipSynchronizer.java | 13 +- .../streamprocessing/ReaderGroupPruner.java | 85 +++++++++ 4 files changed, 303 insertions(+), 16 deletions(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index fd1205c2..e337392a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -1,39 +1,62 @@ package io.pravega.example.streamprocessing; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroup; +import io.pravega.client.stream.Serializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledExecutorService; abstract public class AtLeastOnceProcessor implements Callable { - private static final Logger log = LoggerFactory.getLogger(NonRecoverableSingleThreadedProcessor.class); + private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); + private final ReaderGroup readerGroup; + private final Serializer serializer; + private final ReaderConfig readerConfig; + private final EventStreamClientFactory eventStreamClientFactory; + private final SynchronizerClientFactory synchronizerClientFactory; + private final ScheduledExecutorService executor; + private final long heartbeatIntervalMillis; private final long readTimeoutMillis; - public AtLeastOnceProcessor() { - readTimeoutMillis = 1000; - } + + @Override public Void call() throws Exception { // TODO: handle case when reader dies (if checkpoint timeout occurs, reset reader group to last successful checkpoint) - try (final EventStreamReader reader = createReader()) { - for (; ; ) { - final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); - log.info("call: eventRead={}", eventRead); - if (eventRead.isCheckpoint()) { - flush(eventRead); - } else if (eventRead.getEvent() != null) { - write(eventRead); + final String readerId = UUID.randomUUID().toString(); + try (final ReaderGroupPruner pruner = ReaderGroupPruner.create( + readerGroup, + readerId, + synchronizerClientFactory, + executor, + heartbeatIntervalMillis)) { + try (final EventStreamReader reader = eventStreamClientFactory.createReader( + readerId, + readerGroup.getGroupName(), + serializer, + readerConfig)) { + for (; ; ) { + final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); + log.info("call: eventRead={}", eventRead); + if (eventRead.isCheckpoint()) { + flush(eventRead); + } else if (eventRead.getEvent() != null) { + write(eventRead); + } } } } } - abstract public EventStreamReader createReader(); - abstract public void write(EventRead eventRead); public void flush(EventRead eventRead) { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java new file mode 100644 index 00000000..6ca93663 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java @@ -0,0 +1,170 @@ +/* + * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * and writing each output event to another Pravega stream. + * + * This runs only a single thread. + * + * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} + * to view the output events. + * + * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. + */ +public class AtLeastOnceProcessorMain { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(AtLeastOnceProcessorMain.class); + + private static final int READER_TIMEOUT_MS = 2000; + + private final String scope; + private final String inputStreamName; + private final String outputStreamName; + private final URI controllerURI; + + public AtLeastOnceProcessorMain(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { + this.scope = scope; + this.inputStreamName = inputStreamName; + this.outputStreamName = outputStreamName; + this.controllerURI = controllerURI; + } + + public static void main(String[] args) throws Exception { + AtLeastOnceProcessorMain processor = new AtLeastOnceProcessorMain( + Parameters.getScope(), + Parameters.getStream1Name(), + Parameters.getStream2Name(), + Parameters.getControllerURI()); + processor.run(); + } + + public void run() throws Exception { + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); + try (StreamManager streamManager = StreamManager.create(controllerURI)) { + streamManager.createScope(scope); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + Parameters.getTargetRateEventsPerSec(), + Parameters.getScaleFactor(), + Parameters.getMinNumSegments())) + .build(); + streamManager.createStream(scope, inputStreamName, streamConfig); + streamManager.createStream(scope, outputStreamName, streamConfig); + } + + // Create a reader group that begins at the earliest event. + final String readerGroup = UUID.randomUUID().toString().replace("-", ""); + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, inputStreamName)) + .build(); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + } + + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig)) { + final AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { + @Override + public EventStreamReader createReader() { + return clientFactory.createReader( + readerId, + readerGroup, + new UTF8StringSerializer(), + ReaderConfig.builder().build());; + } + + @Override + public void write(EventRead eventRead) { + + } + }; + processor.call(); + } + + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new UTF8StringSerializer(), + ReaderConfig.builder().build()); + EventStreamWriter writer = clientFactory.createEventWriter( + outputStreamName, + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { + + long eventCounter = 0; + + for (; ; ) { + // Read input event. + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + log.debug("readEvents: eventRead={}", eventRead); + + if (eventRead.getEvent() != null) { + eventCounter++; + log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); + + // Parse input event. + String[] cols = eventRead.getEvent().split(","); + long generatedEventCounter = Long.parseLong(cols[0]); + String routingKey = cols[1]; + long intData = Long.parseLong(cols[2]); + long generatedSum = Long.parseLong(cols[3]); + String generatedTimestampStr = cols[4]; + + // Process the input event. + String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); + + // Build the output event. + String message = String.join(",", + String.format("%06d", generatedEventCounter), + String.format("%06d", eventCounter), + routingKey, + String.format("%02d", intData), + String.format("%08d", generatedSum), + String.format("%03d", 0), + generatedTimestampStr, + processedTimestampStr, + ""); + + // Write the output event. + log.info("eventCounter={}, event={}", + String.format("%06d", eventCounter), + message); + // Note that writeEvent returns a future. When the event has been durably persisted + // to Pravega, the future will complete. + final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + } + } + } + } +} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index aeb5e0d2..d37b5645 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -58,7 +58,7 @@ public class MembershipSynchronizer extends AbstractService { */ private static final int DEATH_THRESHOLD = 10; - private final String instanceId = UUID.randomUUID().toString(); + private final String instanceId; private final AtomicBoolean healthy = new AtomicBoolean(); @@ -68,11 +68,12 @@ public class MembershipSynchronizer extends AbstractService { private final MembershipListener listener; private ScheduledFuture task; - MembershipSynchronizer(String streamName, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, + MembershipSynchronizer(String streamName, String instanceId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, MembershipListener listener) { Preconditions.checkNotNull(streamName); Preconditions.checkNotNull(clientFactory); Preconditions.checkNotNull(listener); + this.instanceId = instanceId; this.executor = executor; this.listener = listener; stateSync = clientFactory.createStateSynchronizer(streamName, @@ -218,6 +219,7 @@ public boolean isCurrentlyHealthy() { } public Set getCurrentMembers() { + stateSync.fetchUpdates(); return stateSync.getState().getLiveInstances(); } @@ -229,10 +231,17 @@ public interface MembershipListener { @Override protected void doStart() { + // Try to ensure that this instance is considered healthy before returning. + stateSync.fetchUpdates(); + stateSync.updateStateUnconditionally(new HeartBeat(instanceId, stateSync.getState().vectorTime)); + stateSync.fetchUpdates(); + notifyListener(); + task = executor.scheduleAtFixedRate(new HeartBeater(), UPDATE_INTERVAL_MILLIS, UPDATE_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); + notifyStarted(); } @Override diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java new file mode 100644 index 00000000..86ce4530 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -0,0 +1,85 @@ +package io.pravega.example.streamprocessing; + +import com.google.common.util.concurrent.AbstractService; +import io.pravega.client.SynchronizerClientFactory; +import io.pravega.client.stream.ReaderGroup; +import lombok.extern.slf4j.Slf4j; + +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class ReaderGroupPruner extends AbstractService implements AutoCloseable { + private final ReaderGroup readerGroup; + private final MembershipSynchronizer membershipSynchronizer; + private final ScheduledExecutorService executor; + private final long heartbeatIntervalMillis; + + private ScheduledFuture task; + + public static ReaderGroupPruner create(ReaderGroup readerGroup, String readerId, SynchronizerClientFactory clientFactory, + ScheduledExecutorService executor, long heartbeatIntervalMillis) { + final ReaderGroupPruner pruner = new ReaderGroupPruner(readerGroup, readerId, clientFactory, executor, heartbeatIntervalMillis); + pruner.startAsync(); + pruner.awaitRunning(); + return pruner; + } + + public ReaderGroupPruner(ReaderGroup readerGroup, String readerId, SynchronizerClientFactory clientFactory, + ScheduledExecutorService executor, long heartbeatIntervalMillis) { + this.readerGroup = readerGroup; + + final String membershipSynchronizerStreamName = readerGroup.getGroupName() + "-membership"; + + // No-op listener + final MembershipSynchronizer.MembershipListener membershipListener = new MembershipSynchronizer.MembershipListener() { + @Override + public void healthy() { + } + + @Override + public void unhealthy() { + } + }; + this.membershipSynchronizer = new MembershipSynchronizer(membershipSynchronizerStreamName, readerId, clientFactory, executor, membershipListener); + this.executor = executor; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + } + + private class PruneRunner implements Runnable { + @Override + public void run() { + try { + Set rgMembers = readerGroup.getOnlineReaders(); + Set msMembers = membershipSynchronizer.getCurrentMembers(); + rgMembers.removeAll(msMembers); + rgMembers.forEach(readerId -> readerGroup.readerOffline(readerId, null)); + } catch (Exception e) { + log.warn("Encountered an error while pruning reader group {}: ", readerGroup.getGroupName(), e); + // Ignore error. It will retry at the next iteration. + } + } + } + + @Override + protected void doStart() { + // Must ensure that we add this reader to MS before RG. + membershipSynchronizer.startAsync(); + membershipSynchronizer.awaitRunning(); + task = executor.scheduleAtFixedRate(new PruneRunner(), heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); + notifyStarted(); + } + + @Override + protected void doStop() { + task.cancel(false); + membershipSynchronizer.stopAsync(); + } + + @Override + public void close() throws Exception { + stopAsync(); + } +} From adb61b88078ac1e227fe3fe5cd476726bb6c3cac Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 3 Jun 2020 05:49:34 +0000 Subject: [PATCH 12/78] Add ReaderGroupPruner (incomplete) Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 16 ++- .../AtLeastOnceProcessorMain.java | 117 ++++++------------ .../MembershipSynchronizer.java | 4 +- .../streamprocessing/ReaderGroupPruner.java | 11 +- .../pravega/example/StreamProcessingTest.java | 60 ++++----- 5 files changed, 86 insertions(+), 122 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index e337392a..5e4b13a1 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -18,6 +18,7 @@ abstract public class AtLeastOnceProcessor implements Callable { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); private final ReaderGroup readerGroup; + private final String membershipSynchronizerStreamName; private final Serializer serializer; private final ReaderConfig readerConfig; private final EventStreamClientFactory eventStreamClientFactory; @@ -26,15 +27,24 @@ abstract public class AtLeastOnceProcessor implements Callable { private final long heartbeatIntervalMillis; private final long readTimeoutMillis; - - + public AtLeastOnceProcessor(ReaderGroup readerGroup, String membershipSynchronizerStreamName, Serializer serializer, ReaderConfig readerConfig, EventStreamClientFactory eventStreamClientFactory, SynchronizerClientFactory synchronizerClientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis, long readTimeoutMillis) { + this.readerGroup = readerGroup; + this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; + this.serializer = serializer; + this.readerConfig = readerConfig; + this.eventStreamClientFactory = eventStreamClientFactory; + this.synchronizerClientFactory = synchronizerClientFactory; + this.executor = executor; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + this.readTimeoutMillis = readTimeoutMillis; + } @Override public Void call() throws Exception { - // TODO: handle case when reader dies (if checkpoint timeout occurs, reset reader group to last successful checkpoint) final String readerId = UUID.randomUUID().toString(); try (final ReaderGroupPruner pruner = ReaderGroupPruner.create( readerGroup, + membershipSynchronizerStreamName, readerId, synchronizerClientFactory, executor, diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java index 6ca93663..06273069 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java @@ -12,6 +12,7 @@ import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventRead; @@ -19,6 +20,7 @@ import io.pravega.client.stream.EventStreamWriter; import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroup; import io.pravega.client.stream.ReaderGroupConfig; import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; @@ -31,6 +33,7 @@ import java.util.Date; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; /** * A simple example that demonstrates reading events from a Pravega stream, processing each event, @@ -49,12 +52,14 @@ public class AtLeastOnceProcessorMain { private static final int READER_TIMEOUT_MS = 2000; private final String scope; + private final String readerGroupName; private final String inputStreamName; private final String outputStreamName; private final URI controllerURI; - public AtLeastOnceProcessorMain(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { + public AtLeastOnceProcessorMain(String scope, String readerGroupName, String inputStreamName, String outputStreamName, URI controllerURI) { this.scope = scope; + this.readerGroupName = readerGroupName; this.inputStreamName = inputStreamName; this.outputStreamName = outputStreamName; this.controllerURI = controllerURI; @@ -63,6 +68,7 @@ public AtLeastOnceProcessorMain(String scope, String inputStreamName, String out public static void main(String[] args) throws Exception { AtLeastOnceProcessorMain processor = new AtLeastOnceProcessorMain( Parameters.getScope(), + Parameters.getStream1Name() + "-rg", Parameters.getStream1Name(), Parameters.getStream2Name(), Parameters.getControllerURI()); @@ -71,9 +77,10 @@ public static void main(String[] args) throws Exception { public void run() throws Exception { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); - try (StreamManager streamManager = StreamManager.create(controllerURI)) { + final String membershipSynchronizerStreamName = readerGroupName + "-membership"; + try (StreamManager streamManager = StreamManager.create(clientConfig)) { streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() + final StreamConfiguration streamConfig = StreamConfiguration.builder() .scalingPolicy(ScalingPolicy.byEventRate( Parameters.getTargetRateEventsPerSec(), Parameters.getScaleFactor(), @@ -81,89 +88,35 @@ public void run() throws Exception { .build(); streamManager.createStream(scope, inputStreamName, streamConfig); streamManager.createStream(scope, outputStreamName, streamConfig); + streamManager.createStream( + scope, + membershipSynchronizerStreamName, + StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); } - - // Create a reader group that begins at the earliest event. - final String readerGroup = UUID.randomUUID().toString().replace("-", ""); + final String readerGroupName = UUID.randomUUID().toString().replace("-", ""); final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(scope, inputStreamName)) .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { - readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - } - - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig)) { - final AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { - @Override - public EventStreamReader createReader() { - return clientFactory.createReader( - readerId, - readerGroup, - new UTF8StringSerializer(), - ReaderConfig.builder().build());; - } - - @Override - public void write(EventRead eventRead) { - - } - }; - processor.call(); - } - - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); - EventStreamReader reader = clientFactory.createReader( - "reader", - readerGroup, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build())) { - - long eventCounter = 0; - - for (; ; ) { - // Read input event. - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - log.debug("readEvents: eventRead={}", eventRead); - - if (eventRead.getEvent() != null) { - eventCounter++; - log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); - - // Parse input event. - String[] cols = eventRead.getEvent().split(","); - long generatedEventCounter = Long.parseLong(cols[0]); - String routingKey = cols[1]; - long intData = Long.parseLong(cols[2]); - long generatedSum = Long.parseLong(cols[3]); - String generatedTimestampStr = cols[4]; - - // Process the input event. - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - String.format("%06d", generatedEventCounter), - String.format("%06d", eventCounter), - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedSum), - String.format("%03d", 0), - generatedTimestampStr, - processedTimestampStr, - ""); - - // Write the output event. - log.info("eventCounter={}, event={}", - String.format("%06d", eventCounter), - message); - // Note that writeEvent returns a future. When the event has been durably persisted - // to Pravega, the future will complete. - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); - } + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { + readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); + final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(readerGroupName); + try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(scope, clientConfig)) { + final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + readerGroup, + membershipSynchronizerStreamName, + new UTF8StringSerializer(), + ReaderConfig.builder().build(), + eventStreamClientFactory, + synchronizerClientFactory, + Executors.newScheduledThreadPool(1), + 500, + 1000) { + @Override + public void write(EventRead eventRead) { + } + }; + processor.call(); } } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index d37b5645..9273dae2 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -16,6 +16,7 @@ import io.pravega.client.state.Revision; import io.pravega.client.state.Revisioned; import io.pravega.client.state.StateSynchronizer; +import io.pravega.client.state.SynchronizerConfig; import io.pravega.client.state.Update; import io.pravega.client.stream.impl.JavaSerializer; import lombok.Data; @@ -79,7 +80,7 @@ public class MembershipSynchronizer extends AbstractService { stateSync = clientFactory.createStateSynchronizer(streamName, new JavaSerializer(), new JavaSerializer(), - null); + SynchronizerConfig.builder().build()); } @Data @@ -233,6 +234,7 @@ public interface MembershipListener { protected void doStart() { // Try to ensure that this instance is considered healthy before returning. stateSync.fetchUpdates(); + Preconditions.checkNotNull(stateSync.getState()); stateSync.updateStateUnconditionally(new HeartBeat(instanceId, stateSync.getState().vectorTime)); stateSync.fetchUpdates(); notifyListener(); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 86ce4530..bee5d7f9 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -19,20 +19,18 @@ public class ReaderGroupPruner extends AbstractService implements AutoCloseable private ScheduledFuture task; - public static ReaderGroupPruner create(ReaderGroup readerGroup, String readerId, SynchronizerClientFactory clientFactory, - ScheduledExecutorService executor, long heartbeatIntervalMillis) { - final ReaderGroupPruner pruner = new ReaderGroupPruner(readerGroup, readerId, clientFactory, executor, heartbeatIntervalMillis); + public static ReaderGroupPruner create(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis) { + final ReaderGroupPruner pruner = new ReaderGroupPruner(readerGroup, membershipSynchronizerStreamName, readerId, clientFactory, + executor, heartbeatIntervalMillis); pruner.startAsync(); pruner.awaitRunning(); return pruner; } - public ReaderGroupPruner(ReaderGroup readerGroup, String readerId, SynchronizerClientFactory clientFactory, + public ReaderGroupPruner(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis) { this.readerGroup = readerGroup; - final String membershipSynchronizerStreamName = readerGroup.getGroupName() + "-membership"; - // No-op listener final MembershipSynchronizer.MembershipListener membershipListener = new MembershipSynchronizer.MembershipListener() { @Override @@ -76,6 +74,7 @@ protected void doStart() { protected void doStop() { task.cancel(false); membershipSynchronizer.stopAsync(); + // TODO: Can we safely delete the membershipSynchronizer stream? } @Override diff --git a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java index 7297f686..947592d9 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java @@ -65,35 +65,35 @@ public void basicTest() throws Exception { .build(); final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); - try (StreamManager streamManager = StreamManager.create(clientConfig)) { - streamManager.createScope(scope); - - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.fixed(3)) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { - readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - } - try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); - final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { - - inputWriter.writeEvent("CLAUDIO1"); - - AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { - @Override - public EventStreamReader createReader() { - return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); - } - - @Override - public void write(EventRead eventRead) { - } - }; - processor.call(); - // TODO: Wait for correct result and then terminate. - } - } +// try (StreamManager streamManager = StreamManager.create(clientConfig)) { +// streamManager.createScope(scope); +// +// StreamConfiguration streamConfig = StreamConfiguration.builder() +// .scalingPolicy(ScalingPolicy.fixed(3)) +// .build(); +// streamManager.createStream(scope, inputStreamName, streamConfig); +// +// try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { +// readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); +// } +// try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); +// final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { +// +// inputWriter.writeEvent("CLAUDIO1"); +// +// AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { +// @Override +// public EventStreamReader createReader() { +// return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); +// } +// +// @Override +// public void write(EventRead eventRead) { +// } +// }; +// processor.call(); +// // TODO: Wait for correct result and then terminate. +// } +// } } } From 6bb05b182ee29db91338ab4cb5b57f7dd1cd2947 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 4 Jun 2020 01:06:06 +0000 Subject: [PATCH 13/78] Bug fixes in MembershipSynchronizer Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessorMain.java | 1 - .../MembershipSynchronizer.java | 34 ++++++++++++++----- .../streamprocessing/ReaderGroupPruner.java | 27 +++++++-------- 3 files changed, 39 insertions(+), 23 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java index 06273069..47c07450 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java @@ -93,7 +93,6 @@ public void run() throws Exception { membershipSynchronizerStreamName, StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); } - final String readerGroupName = UUID.randomUUID().toString().replace("-", ""); final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(scope, inputStreamName)) .build(); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 9273dae2..0812b34e 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -134,13 +134,26 @@ public Set getLiveInstances() { } } + private static class CreateState implements Serializable, InitialUpdate { + private static final long serialVersionUID = 1L; + + @Override + public LiveInstances create(String scopedStreamName, Revision revision) { + return new LiveInstances(scopedStreamName, revision, new HashMap<>(), 0); + } + } + private class HeartBeater implements Runnable { @Override public void run() { try { stateSync.fetchUpdates(); notifyListener(); - if (stateSync.getState().isOverUnconditionalThreshold(instanceId)) { + log.info("run: BEGIN: vectorTime={}, isOverUnconditionalThreshold={}, liveInstances={}", + stateSync.getState().getVectorTime(), + stateSync.getState().isOverUnconditionalThreshold(instanceId), + stateSync.getState().liveInstances); + if (!stateSync.getState().isOverUnconditionalThreshold(instanceId)) { stateSync.updateState((state, updates) -> { long vectorTime = state.getVectorTime() + 1; updates.add(new HeartBeat(instanceId, vectorTime)); @@ -155,8 +168,11 @@ public void run() { stateSync.fetchUpdates(); } notifyListener(); + log.info("run: END: vectorTime={}, liveInstances={}", + stateSync.getState().getVectorTime(), + stateSync.getState().liveInstances); } catch (Exception e) { - log.warn("Encountered an error while heartbeating: " + e); + log.warn("Encountered an error while heartbeating", e); if (healthy.compareAndSet(true, false)) { listener.unhealthy(); } @@ -164,12 +180,12 @@ public void run() { } } - private abstract class HeartbeatUpdate implements Update, Serializable { + private static abstract class HeartbeatUpdate implements Update, Serializable { private static final long serialVersionUID = 1L; } @RequiredArgsConstructor - private final class HeartBeat extends HeartbeatUpdate { + private static class HeartBeat extends HeartbeatUpdate { private static final long serialVersionUID = 1L; private final String name; private final long timestamp; @@ -177,7 +193,7 @@ private final class HeartBeat extends HeartbeatUpdate { @Override public LiveInstances applyTo(LiveInstances state, Revision newRevision) { Map timestamps = new HashMap<>(state.liveInstances); - long vectorTime = Long.max(timestamps.values().stream().max(Long::compare).get(), timestamp); + long vectorTime = Long.max(timestamps.values().stream().max(Long::compare).orElse(0L), timestamp); timestamps.put(name, timestamp); return new LiveInstances(state.scopedStreamName, newRevision, @@ -187,7 +203,7 @@ public LiveInstances applyTo(LiveInstances state, Revision newRevision) { } @RequiredArgsConstructor - private final class DeclareDead extends HeartbeatUpdate { + private static final class DeclareDead extends HeartbeatUpdate { private static final long serialVersionUID = 1L; private final String name; @@ -225,13 +241,15 @@ public Set getCurrentMembers() { } public interface MembershipListener { - void healthy(); + default void healthy() {}; - void unhealthy(); + default void unhealthy() {}; } @Override protected void doStart() { + // Create initial empty state if stream is empty. + stateSync.initialize(new CreateState()); // Try to ensure that this instance is considered healthy before returning. stateSync.fetchUpdates(); Preconditions.checkNotNull(stateSync.getState()); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index bee5d7f9..881a89cb 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -30,18 +30,12 @@ public static ReaderGroupPruner create(ReaderGroup readerGroup, String membershi public ReaderGroupPruner(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis) { this.readerGroup = readerGroup; - - // No-op listener - final MembershipSynchronizer.MembershipListener membershipListener = new MembershipSynchronizer.MembershipListener() { - @Override - public void healthy() { - } - - @Override - public void unhealthy() { - } - }; - this.membershipSynchronizer = new MembershipSynchronizer(membershipSynchronizerStreamName, readerId, clientFactory, executor, membershipListener); + this.membershipSynchronizer = new MembershipSynchronizer( + membershipSynchronizerStreamName, + readerId, + clientFactory, + executor, + new MembershipSynchronizer.MembershipListener() {}); this.executor = executor; this.heartbeatIntervalMillis = heartbeatIntervalMillis; } @@ -52,10 +46,15 @@ public void run() { try { Set rgMembers = readerGroup.getOnlineReaders(); Set msMembers = membershipSynchronizer.getCurrentMembers(); + log.info("rgMembers={}", rgMembers); + log.info("msMembers={}", msMembers); rgMembers.removeAll(msMembers); - rgMembers.forEach(readerId -> readerGroup.readerOffline(readerId, null)); + rgMembers.forEach(readerId -> { + log.info("Removing dead reader {} from reader group {}", readerId, readerGroup.getGroupName()); + readerGroup.readerOffline(readerId, null); + }); } catch (Exception e) { - log.warn("Encountered an error while pruning reader group {}: ", readerGroup.getGroupName(), e); + log.warn("Encountered an error while pruning reader group {}", readerGroup.getGroupName(), e); // Ignore error. It will retry at the next iteration. } } From 9584ce082696767cc136db6f3d447d0e98ade823 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 01:47:38 +0000 Subject: [PATCH 14/78] Document new classes. Add various parameters. Signed-off-by: Claudio Fahey --- ...ProcessorMain.java => AtLeastOnceApp.java} | 43 +++-- .../MembershipSynchronizer.java | 30 ++-- ...NonRecoverableSingleThreadedProcessor.java | 151 ------------------ .../example/streamprocessing/Parameters.java | 20 ++- .../streamprocessing/ReaderGroupPruner.java | 7 +- 5 files changed, 57 insertions(+), 194 deletions(-) rename pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/{AtLeastOnceProcessorMain.java => AtLeastOnceApp.java} (76%) delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java similarity index 76% rename from pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java rename to pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 47c07450..6e6d271d 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorMain.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -16,9 +16,6 @@ import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventRead; -import io.pravega.client.stream.EventStreamReader; -import io.pravega.client.stream.EventStreamWriter; -import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ReaderConfig; import io.pravega.client.stream.ReaderGroup; import io.pravega.client.stream.ReaderGroupConfig; @@ -29,55 +26,55 @@ import org.slf4j.LoggerFactory; import java.net.URI; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; /** - * A simple example that demonstrates reading events from a Pravega stream, processing each event, + * This demonstrates reading events from a Pravega stream, processing each event, * and writing each output event to another Pravega stream. - * - * This runs only a single thread. + * It guarantees that each event is processed at least once. + * If multiple instances of this application are executed using the same readerGroupName parameter, + * each instance will get a distinct subset of events. * * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} * to view the output events. - * - * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. */ -public class AtLeastOnceProcessorMain { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(AtLeastOnceProcessorMain.class); - - private static final int READER_TIMEOUT_MS = 2000; +public class AtLeastOnceApp { + private static final org.slf4j.Logger log = LoggerFactory.getLogger(AtLeastOnceApp.class); private final String scope; private final String readerGroupName; + private final String membershipSynchronizerStreamName; private final String inputStreamName; private final String outputStreamName; private final URI controllerURI; + private final long heartbeatIntervalMillis; - public AtLeastOnceProcessorMain(String scope, String readerGroupName, String inputStreamName, String outputStreamName, URI controllerURI) { + public AtLeastOnceApp(String scope, String readerGroupName, String membershipSynchronizerStreamName, + String inputStreamName, String outputStreamName, URI controllerURI, + long heartbeatIntervalMillis) { this.scope = scope; this.readerGroupName = readerGroupName; + this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; this.inputStreamName = inputStreamName; this.outputStreamName = outputStreamName; this.controllerURI = controllerURI; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; } public static void main(String[] args) throws Exception { - AtLeastOnceProcessorMain processor = new AtLeastOnceProcessorMain( + AtLeastOnceApp processor = new AtLeastOnceApp( Parameters.getScope(), - Parameters.getStream1Name() + "-rg", + Parameters.getReaderGroup(), + Parameters.getMembershipSynchronizerStreamName(), Parameters.getStream1Name(), Parameters.getStream2Name(), - Parameters.getControllerURI()); + Parameters.getControllerURI(), + Parameters.getHeartbeatIntervalMillis()); processor.run(); } public void run() throws Exception { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); - final String membershipSynchronizerStreamName = readerGroupName + "-membership"; try (StreamManager streamManager = StreamManager.create(clientConfig)) { streamManager.createScope(scope); final StreamConfiguration streamConfig = StreamConfiguration.builder() @@ -88,6 +85,7 @@ public void run() throws Exception { .build(); streamManager.createStream(scope, inputStreamName, streamConfig); streamManager.createStream(scope, outputStreamName, streamConfig); + // Create stream for the membership state synchronizer. streamManager.createStream( scope, membershipSynchronizerStreamName, @@ -95,6 +93,7 @@ public void run() throws Exception { } final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(scope, inputStreamName)) + .automaticCheckpointIntervalMillis(Parameters.getCheckpointPeriodMs()) .build(); try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); @@ -109,7 +108,7 @@ public void run() throws Exception { eventStreamClientFactory, synchronizerClientFactory, Executors.newScheduledThreadPool(1), - 500, + heartbeatIntervalMillis, 1000) { @Override public void write(EventRead eventRead) { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 0812b34e..358959ab 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -29,21 +29,21 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; +/** + * This class maintains a list of process names for processes that are healthy, + * as indicated by periodic heartbeat messages. + * It uses a Pravega state synchronizer to allow processes to communicate + * with each other. + */ @Slf4j public class MembershipSynchronizer extends AbstractService { - /** - * How frequently to update the segment using a heartbeat. - */ - private static final int UPDATE_INTERVAL_MILLIS = 1000; - /** * Number of intervals behind before switching to unconditional updates. */ @@ -61,6 +61,11 @@ public class MembershipSynchronizer extends AbstractService { private final String instanceId; + /** + * How frequently to update the segment using a heartbeat. + */ + private final long heartbeatIntervalMillis; + private final AtomicBoolean healthy = new AtomicBoolean(); private final ScheduledExecutorService executor; @@ -69,12 +74,13 @@ public class MembershipSynchronizer extends AbstractService { private final MembershipListener listener; private ScheduledFuture task; - MembershipSynchronizer(String streamName, String instanceId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, + MembershipSynchronizer(String streamName, String instanceId, long heartbeatIntervalMillis, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, MembershipListener listener) { Preconditions.checkNotNull(streamName); Preconditions.checkNotNull(clientFactory); Preconditions.checkNotNull(listener); this.instanceId = instanceId; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; this.executor = executor; this.listener = listener; stateSync = clientFactory.createStateSynchronizer(streamName, @@ -252,15 +258,15 @@ protected void doStart() { stateSync.initialize(new CreateState()); // Try to ensure that this instance is considered healthy before returning. stateSync.fetchUpdates(); - Preconditions.checkNotNull(stateSync.getState()); stateSync.updateStateUnconditionally(new HeartBeat(instanceId, stateSync.getState().vectorTime)); stateSync.fetchUpdates(); notifyListener(); - task = executor.scheduleAtFixedRate(new HeartBeater(), - UPDATE_INTERVAL_MILLIS, - UPDATE_INTERVAL_MILLIS, - TimeUnit.MILLISECONDS); + task = executor.scheduleAtFixedRate( + new HeartBeater(), + heartbeatIntervalMillis, + heartbeatIntervalMillis, + TimeUnit.MILLISECONDS); notifyStarted(); } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java deleted file mode 100644 index 3fe6708f..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/NonRecoverableSingleThreadedProcessor.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. - * - * Licensed 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 - * - */ -package io.pravega.example.streamprocessing; - -import io.pravega.client.ClientConfig; -import io.pravega.client.EventStreamClientFactory; -import io.pravega.client.admin.ReaderGroupManager; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.EventRead; -import io.pravega.client.stream.EventStreamReader; -import io.pravega.client.stream.EventStreamWriter; -import io.pravega.client.stream.EventWriterConfig; -import io.pravega.client.stream.ReaderConfig; -import io.pravega.client.stream.ReaderGroupConfig; -import io.pravega.client.stream.ScalingPolicy; -import io.pravega.client.stream.Stream; -import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import org.slf4j.LoggerFactory; - -import java.net.URI; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; - -/** - * A simple example that demonstrates reading events from a Pravega stream, processing each event, - * and writing each output event to another Pravega stream. - * - * This runs only a single thread. - * - * Use {@link EventGenerator} to generate input events and {@link EventDebugSink} - * to view the output events. - * - * See {@link ExactlyOnceMultithreadedProcessor} for an improved version. - */ -public class NonRecoverableSingleThreadedProcessor { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(NonRecoverableSingleThreadedProcessor.class); - - private static final int READER_TIMEOUT_MS = 2000; - - private final String scope; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - - public NonRecoverableSingleThreadedProcessor(String scope, String inputStreamName, String outputStreamName, URI controllerURI) { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - } - - public static void main(String[] args) throws Exception { - NonRecoverableSingleThreadedProcessor processor = new NonRecoverableSingleThreadedProcessor( - Parameters.getScope(), - Parameters.getStream1Name(), - Parameters.getStream2Name(), - Parameters.getControllerURI()); - processor.run(); - } - - public void run() throws Exception { - final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) - .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - streamManager.createStream(scope, outputStreamName, streamConfig); - } - - // Create a reader group that begins at the earliest event. - final String readerGroup = UUID.randomUUID().toString().replace("-", ""); - final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() - .stream(Stream.of(scope, inputStreamName)) - .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { - readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - } - - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); - EventStreamReader reader = clientFactory.createReader( - "reader", - readerGroup, - new UTF8StringSerializer(), - ReaderConfig.builder().build()); - EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, - new UTF8StringSerializer(), - EventWriterConfig.builder().build())) { - - long eventCounter = 0; - - for (; ; ) { - // Read input event. - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - log.debug("readEvents: eventRead={}", eventRead); - - if (eventRead.getEvent() != null) { - eventCounter++; - log.debug("Read eventCounter={}, event={}", String.format("%06d", eventCounter), eventRead.getEvent()); - - // Parse input event. - String[] cols = eventRead.getEvent().split(","); - long generatedEventCounter = Long.parseLong(cols[0]); - String routingKey = cols[1]; - long intData = Long.parseLong(cols[2]); - long generatedSum = Long.parseLong(cols[3]); - String generatedTimestampStr = cols[4]; - - // Process the input event. - String processedTimestampStr = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new Date()); - - // Build the output event. - String message = String.join(",", - String.format("%06d", generatedEventCounter), - String.format("%06d", eventCounter), - routingKey, - String.format("%02d", intData), - String.format("%08d", generatedSum), - String.format("%03d", 0), - generatedTimestampStr, - processedTimestampStr, - ""); - - // Write the output event. - log.info("eventCounter={}, event={}", - String.format("%06d", eventCounter), - message); - // Note that writeEvent returns a future. When the event has been durably persisted - // to Pravega, the future will complete. - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); - } - } - } - } -} diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java index fc951574..b2baf914 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java @@ -26,6 +26,14 @@ public static String getScope() { return getEnvVar("PRAVEGA_SCOPE", "examples"); } + public static String getReaderGroup() { + return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing-rg1"); + } + + public static String getMembershipSynchronizerStreamName() { + return getReaderGroup() + "-membership"; + } + public static String getStream1Name() { return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1"); } @@ -46,14 +54,6 @@ public static int getMinNumSegments() { return Integer.parseInt(getEnvVar("PRAVEGA_MIN_NUM_SEGMENTS", "3")); } - public static int getNumWorkers() { - return Integer.parseInt(getEnvVar("NUM_WORKERS", "2")); - } - - public static Path getCheckpointRootPath() { - return Paths.get(getEnvVar("CHECKPOINT_ROOT_PATH", "/tmp/checkpoints")); - } - public static long getCheckpointPeriodMs() { return Long.parseLong(getEnvVar("CHECKPOINT_PERIOD_MS", "3000")); } @@ -66,6 +66,10 @@ public static long getTransactionTimeoutMs() { return Long.parseLong(getEnvVar("TRANSACTION_TIMEOUT_MS", "120000")); } + public static long getHeartbeatIntervalMillis() { + return Long.parseLong(getEnvVar("HEARTBEAT_INTERVAL_MS", "500")); + } + private static String getEnvVar(String name, String defaultValue) { String value = System.getenv(name); if (value == null || value.isEmpty()) { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 881a89cb..d0baa818 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -10,6 +10,10 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +/** + * This class removes unhealthy processes from a Pravega reader group. + * It uses {@link MembershipSynchronizer} to identify healthy processes. + */ @Slf4j public class ReaderGroupPruner extends AbstractService implements AutoCloseable { private final ReaderGroup readerGroup; @@ -33,6 +37,7 @@ public ReaderGroupPruner(ReaderGroup readerGroup, String membershipSynchronizerS this.membershipSynchronizer = new MembershipSynchronizer( membershipSynchronizerStreamName, readerId, + heartbeatIntervalMillis, clientFactory, executor, new MembershipSynchronizer.MembershipListener() {}); @@ -62,7 +67,7 @@ public void run() { @Override protected void doStart() { - // Must ensure that we add this reader to MS before RG. + // MWe must ensure that we add this reader to the membership synchronizer before the reader group. membershipSynchronizer.startAsync(); membershipSynchronizer.awaitRunning(); task = executor.scheduleAtFixedRate(new PruneRunner(), heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); From 4f111778d6d17dbb9bcc1a5917f0add1b3c55d76 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 02:23:49 +0000 Subject: [PATCH 15/78] Use new AppConfiguration class. Update license. Mark obsolete sections in README.md. Signed-off-by: Claudio Fahey --- ...{Parameters.java => AppConfiguration.java} | 41 +++++----- .../streamprocessing/AtLeastOnceApp.java | 78 +++++++------------ .../AtLeastOnceProcessor.java | 10 +++ .../streamprocessing/EventDebugSink.java | 48 +++++------- .../streamprocessing/EventGenerator.java | 48 +++++------- .../MembershipSynchronizer.java | 5 +- .../example/streamprocessing/README.md | 13 ++-- .../streamprocessing/ReaderGroupPruner.java | 10 +++ 8 files changed, 123 insertions(+), 130 deletions(-) rename pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/{Parameters.java => AppConfiguration.java} (65%) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java similarity index 65% rename from pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java rename to pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java index b2baf914..e263647c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Parameters.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -11,62 +11,65 @@ package io.pravega.example.streamprocessing; import java.net.URI; -import java.nio.file.Path; -import java.nio.file.Paths; -// All parameters will come from environment variables. This makes it easy -// to configure on Docker, Kubernetes, etc. -class Parameters { +/** + * All parameters will come from environment variables. + * This makes it easy to configure on Docker, Kubernetes, etc. + */ +class AppConfiguration { + AppConfiguration(String[] args) { + } + // By default, we will connect to a standalone Pravega running on localhost. - public static URI getControllerURI() { + public URI getControllerURI() { return URI.create(getEnvVar("PRAVEGA_CONTROLLER", "tcp://localhost:9090")); } - public static String getScope() { + public String getScope() { return getEnvVar("PRAVEGA_SCOPE", "examples"); } - public static String getReaderGroup() { + public String getReaderGroup() { return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing-rg1"); } - public static String getMembershipSynchronizerStreamName() { + public String getMembershipSynchronizerStreamName() { return getReaderGroup() + "-membership"; } - public static String getStream1Name() { + public String getStream1Name() { return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1"); } - public static String getStream2Name() { + public String getStream2Name() { return getEnvVar("PRAVEGA_STREAM_2", "streamprocessing2"); } - public static int getTargetRateEventsPerSec() { + public int getTargetRateEventsPerSec() { return Integer.parseInt(getEnvVar("PRAVEGA_TARGET_RATE_EVENTS_PER_SEC", "10")); } - public static int getScaleFactor() { + public int getScaleFactor() { return Integer.parseInt(getEnvVar("PRAVEGA_SCALE_FACTOR", "2")); } - public static int getMinNumSegments() { + public int getMinNumSegments() { return Integer.parseInt(getEnvVar("PRAVEGA_MIN_NUM_SEGMENTS", "3")); } - public static long getCheckpointPeriodMs() { + public long getCheckpointPeriodMs() { return Long.parseLong(getEnvVar("CHECKPOINT_PERIOD_MS", "3000")); } - public static long getCheckpointTimeoutMs() { + public long getCheckpointTimeoutMs() { return Long.parseLong(getEnvVar("CHECKPOINT_TIMEOUT_MS", "120000")); } - public static long getTransactionTimeoutMs() { + public long getTransactionTimeoutMs() { return Long.parseLong(getEnvVar("TRANSACTION_TIMEOUT_MS", "120000")); } - public static long getHeartbeatIntervalMillis() { + public long getHeartbeatIntervalMillis() { return Long.parseLong(getEnvVar("HEARTBEAT_INTERVAL_MS", "500")); } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 6e6d271d..97b90c42 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,7 +25,6 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; -import java.net.URI; import java.util.concurrent.Executors; /** @@ -40,75 +39,58 @@ */ public class AtLeastOnceApp { private static final org.slf4j.Logger log = LoggerFactory.getLogger(AtLeastOnceApp.class); + + private final AppConfiguration config; - private final String scope; - private final String readerGroupName; - private final String membershipSynchronizerStreamName; - private final String inputStreamName; - private final String outputStreamName; - private final URI controllerURI; - private final long heartbeatIntervalMillis; - - public AtLeastOnceApp(String scope, String readerGroupName, String membershipSynchronizerStreamName, - String inputStreamName, String outputStreamName, URI controllerURI, - long heartbeatIntervalMillis) { - this.scope = scope; - this.readerGroupName = readerGroupName; - this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; - this.inputStreamName = inputStreamName; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; - this.heartbeatIntervalMillis = heartbeatIntervalMillis; + public static void main(String[] args) throws Exception { + AtLeastOnceApp app = new AtLeastOnceApp(new AppConfiguration(args)); + app.run(); } - public static void main(String[] args) throws Exception { - AtLeastOnceApp processor = new AtLeastOnceApp( - Parameters.getScope(), - Parameters.getReaderGroup(), - Parameters.getMembershipSynchronizerStreamName(), - Parameters.getStream1Name(), - Parameters.getStream2Name(), - Parameters.getControllerURI(), - Parameters.getHeartbeatIntervalMillis()); - processor.run(); + public AtLeastOnceApp(AppConfiguration config) { + this.config = config; } + public AppConfiguration getConfig() { + return config; + } + public void run() throws Exception { - final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); try (StreamManager streamManager = StreamManager.create(clientConfig)) { - streamManager.createScope(scope); + streamManager.createScope(getConfig().getScope()); final StreamConfiguration streamConfig = StreamConfiguration.builder() .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); - streamManager.createStream(scope, outputStreamName, streamConfig); + streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); + streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); // Create stream for the membership state synchronizer. streamManager.createStream( - scope, - membershipSynchronizerStreamName, + getConfig().getScope(), + getConfig().getMembershipSynchronizerStreamName(), StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); } final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() - .stream(Stream.of(scope, inputStreamName)) - .automaticCheckpointIntervalMillis(Parameters.getCheckpointPeriodMs()) + .stream(Stream.of(getConfig().getScope(), getConfig().getStream1Name())) + .automaticCheckpointIntervalMillis(getConfig().getCheckpointPeriodMs()) .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { - readerGroupManager.createReaderGroup(readerGroupName, readerGroupConfig); - final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(readerGroupName); - try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(scope, clientConfig); - SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(scope, clientConfig)) { + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig)) { + readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); + final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); + try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig)) { final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( readerGroup, - membershipSynchronizerStreamName, + getConfig().getMembershipSynchronizerStreamName(), new UTF8StringSerializer(), ReaderConfig.builder().build(), eventStreamClientFactory, synchronizerClientFactory, Executors.newScheduledThreadPool(1), - heartbeatIntervalMillis, + getConfig().getHeartbeatIntervalMillis(), 1000) { @Override public void write(EventRead eventRead) { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 5e4b13a1..08dceeb2 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import io.pravega.client.EventStreamClientFactory; diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 91da8fd7..7d4775db 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,60 +24,54 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; -import java.net.URI; import java.util.UUID; /** * A simple example that continuously shows the events in a stream. - * - * This reads the output of {@link ExactlyOnceMultithreadedProcessor}. */ public class EventDebugSink { private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventDebugSink.class); private static final int READER_TIMEOUT_MS = 2000; - public final String scope; - public final String inputStreamName; - public final URI controllerURI; + private final AppConfiguration config; - public EventDebugSink(String scope, String inputStreamName, URI controllerURI) { - this.scope = scope; - this.inputStreamName = inputStreamName; - this.controllerURI = controllerURI; + public static void main(String[] args) throws Exception { + EventDebugSink app = new EventDebugSink(new AppConfiguration(args)); + app.run(); } - public static void main(String[] args) throws Exception { - EventDebugSink processor = new EventDebugSink( - Parameters.getScope(), - Parameters.getStream2Name(), - Parameters.getControllerURI()); - processor.run(); + public EventDebugSink(AppConfiguration config) { + this.config = config; + } + + public AppConfiguration getConfig() { + return config; } public void run() throws Exception { - final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); + try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { + streamManager.createScope(getConfig().getScope()); StreamConfiguration streamConfig = StreamConfiguration.builder() .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) .build(); - streamManager.createStream(scope, inputStreamName, streamConfig); + streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); } // Create a reader group that begins at the earliest event. final String readerGroup = UUID.randomUUID().toString().replace("-", ""); final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() - .stream(Stream.of(scope, inputStreamName)) + .stream(Stream.of(getConfig().getScope(), getConfig().getStream2Name())) .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, controllerURI)) { + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI())) { readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); } - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); EventStreamReader reader = clientFactory.createReader( "reader", readerGroup, diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 0e8dd6f4..573b8c0c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2017 Dell Inc., or its subsidiaries. All Rights Reserved. + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,55 +20,49 @@ import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; -import java.net.URI; import java.text.SimpleDateFormat; import java.util.Date; import java.util.Random; import java.util.concurrent.CompletableFuture; /** - * A simple example app that to write messages to a Pravega stream. - * - * Use {@link ExactlyOnceMultithreadedProcessor} to read output events. + * A simple example app to write messages to a Pravega stream. */ public class EventGenerator { private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventGenerator.class); - public final String scope; - public final String outputStreamName; - public final URI controllerURI; + private final AppConfiguration config; - public EventGenerator(String scope, String outputStreamName, URI controllerURI) { - this.scope = scope; - this.outputStreamName = outputStreamName; - this.controllerURI = controllerURI; + public static void main(String[] args) throws Exception { + EventGenerator app = new EventGenerator(new AppConfiguration(args)); + app.run(); } - public static void main(String[] args) throws Exception { - EventGenerator processor = new EventGenerator( - Parameters.getScope(), - Parameters.getStream1Name(), - Parameters.getControllerURI()); - processor.run(); + public EventGenerator(AppConfiguration config) { + this.config = config; + } + + public AppConfiguration getConfig() { + return config; } public void run() throws Exception { - final ClientConfig clientConfig = ClientConfig.builder().controllerURI(controllerURI).build(); - try (StreamManager streamManager = StreamManager.create(controllerURI)) { - streamManager.createScope(scope); + final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); + try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { + streamManager.createScope(getConfig().getScope()); StreamConfiguration streamConfig = StreamConfiguration.builder() .scalingPolicy(ScalingPolicy.byEventRate( - Parameters.getTargetRateEventsPerSec(), - Parameters.getScaleFactor(), - Parameters.getMinNumSegments())) + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) .build(); - streamManager.createStream(scope, outputStreamName, streamConfig); + streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); } Random rand = new Random(42); - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); EventStreamWriter writer = clientFactory.createEventWriter( - outputStreamName, + getConfig().getStream1Name(), new UTF8StringSerializer(), EventWriterConfig.builder().build())) { long eventCounter = 0; diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 358959ab..56bcf9b0 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -1,11 +1,12 @@ -/** +/* * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed 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 + * http://www.apache.org/licenses/LICENSE-2.0 + * */ package io.pravega.example.streamprocessing; diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index a90a9a3d..51056d7b 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -1,10 +1,9 @@ # Pravega Stream Processing Example - # Overview The examples in this directory are intended to illustrate -how exactly-once semantics can be achieved with Pravega. +how at-least-once and exactly-once semantics can be achieved with Pravega. To make it clear to the reader, the entirety of the stream processor is defined within this package with minimal dependencies. In particular, these illustrative examples do *not* use Apache Flink. @@ -17,7 +16,7 @@ These examples include: This application generates new events every 1 second and writes them to a Pravega stream (referred to as stream1). -- [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java): +- (OBSOLETE) [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java): This application continuously reads events from stream1, performs a stateless computation to generate output events, and writes the output event to another Pravega stream (referred to as stream2). @@ -55,7 +54,7 @@ These examples include: PRAVEGA_SCOPE=examples PRAVEGA_CONTROLLER=tcp://localhost:9090 ./gradlew pravega-client-examples:startEventGenerator ``` - See [Parameters.java](Parameters.java) for available parameters. + See [Parameters.java](Parameters.java) for available appConfiguration. - In another window, start the stream processor: ``` @@ -67,7 +66,7 @@ These examples include: ./gradlew pravega-client-examples:startEventDebugSink ``` -- Note: The [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java) +- (OBSOLETE) Note: The [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java) will automatically restart from the latest checkpoint. However, if Pravega streams are truncated or deleted, or checkpoint files in /tmp/checkpoints are deleted or otherwise bad, you may need to start over from @@ -77,7 +76,7 @@ These examples include: - Use a new scope (PRAVEGA_SCOPE) or streams (PRAVEGA_STREAM_1 and PRAVEGA_STREAM_2). -# Achieving At-Least-Once Semantics +# (OBSOLETE) Achieving At-Least-Once Semantics The current position in a Pravega stream is defined by a stream cut. A stream cut is essentially a mapping from segment numbers to byte offsets @@ -132,7 +131,7 @@ by the user as a master. There can be any number of worker processes. - Process events as usual. -# Achieving Exactly-Once Semantics +# (OBSOLETE) Achieving Exactly-Once Semantics Exactly-once semantics can achieved by starting with at-least-once semantics and adding write idempotence. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index d0baa818..cb1257be 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import com.google.common.util.concurrent.AbstractService; From 7eac91142650939dd2d567cb9b3942b19cb4b1de Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 03:39:20 +0000 Subject: [PATCH 16/78] Add writer to AtLeastOnceApp Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 20 ++++++++- .../AtLeastOnceProcessor.java | 45 +++++++++++++++++-- .../streamprocessing/EventDebugSink.java | 44 +++++++++--------- .../streamprocessing/ReaderGroupPruner.java | 1 - 4 files changed, 83 insertions(+), 27 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 97b90c42..924c87e5 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -16,6 +16,8 @@ import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ReaderConfig; import io.pravega.client.stream.ReaderGroup; import io.pravega.client.stream.ReaderGroupConfig; @@ -81,7 +83,11 @@ public void run() throws Exception { readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig)) { + SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); + EventStreamWriter writer = eventStreamClientFactory.createEventWriter( + getConfig().getStream2Name(), + new UTF8StringSerializer(), + EventWriterConfig.builder().build())) { final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( readerGroup, getConfig().getMembershipSynchronizerStreamName(), @@ -92,8 +98,18 @@ public void run() throws Exception { Executors.newScheduledThreadPool(1), getConfig().getHeartbeatIntervalMillis(), 1000) { + /** + * Write + * @param eventRead + */ @Override - public void write(EventRead eventRead) { + public void process(EventRead eventRead) { + writer.writeEvent("0", "processed," + eventRead.getEvent()); + } + + @Override + public void flush(EventRead eventRead) { + writer.flush(); } }; processor.call(); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 08dceeb2..e4b54cb5 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -14,6 +14,8 @@ import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.Position; import io.pravega.client.stream.ReaderConfig; import io.pravega.client.stream.ReaderGroup; import io.pravega.client.stream.Serializer; @@ -24,6 +26,14 @@ import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; +/** + * This is an abstract class for implementing a stateless event processor with Pravega. + * It reads an event from a Pravega stream and then calls a user-defined function to process it. + * It guarantees that each event is processed at least once, even if failures occur. + * If multiple instances are executed using the same readerGroup parameter, + * each instance will get a distinct subset of events. + * Instances can be in different processes. + */ abstract public class AtLeastOnceProcessor implements Callable { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); @@ -37,7 +47,16 @@ abstract public class AtLeastOnceProcessor implements Callable { private final long heartbeatIntervalMillis; private final long readTimeoutMillis; - public AtLeastOnceProcessor(ReaderGroup readerGroup, String membershipSynchronizerStreamName, Serializer serializer, ReaderConfig readerConfig, EventStreamClientFactory eventStreamClientFactory, SynchronizerClientFactory synchronizerClientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis, long readTimeoutMillis) { + public AtLeastOnceProcessor( + ReaderGroup readerGroup, + String membershipSynchronizerStreamName, + Serializer serializer, + ReaderConfig readerConfig, + EventStreamClientFactory eventStreamClientFactory, + SynchronizerClientFactory synchronizerClientFactory, + ScheduledExecutorService executor, + long heartbeatIntervalMillis, + long readTimeoutMillis) { this.readerGroup = readerGroup; this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; this.serializer = serializer; @@ -49,6 +68,13 @@ public AtLeastOnceProcessor(ReaderGroup readerGroup, String membershipSynchroniz this.readTimeoutMillis = readTimeoutMillis; } + /** + * Run the event processor loop. + * + * If the previous call to readNextEvent returned a checkpoint, the next call + * to readNextEvent will record in the reader group that this reader + * has read and processed events up to the previous {@link Position}. + */ @Override public Void call() throws Exception { final String readerId = UUID.randomUUID().toString(); @@ -70,15 +96,28 @@ public Void call() throws Exception { if (eventRead.isCheckpoint()) { flush(eventRead); } else if (eventRead.getEvent() != null) { - write(eventRead); + process(eventRead); } } } } } - abstract public void write(EventRead eventRead); + /** + * Process an event that was read. + * Processing can be performed asynchronously after this method returns. + * + * @param eventRead The event read. + */ + abstract public void process(EventRead eventRead); + /** + * This will be called when a checkpoint event is received. + * If {@link #process} did not completely process prior events, it must do so now. + * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. + * + * @param eventRead Identifies the checkpoint name. This can generally be ignored. + */ public void flush(EventRead eventRead) { } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 7d4775db..55ec8b1b 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -69,29 +69,31 @@ public void run() throws Exception { .build(); try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI())) { readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - } - - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamReader reader = clientFactory.createReader( - "reader", - readerGroup, - new UTF8StringSerializer(), - ReaderConfig.builder().build())) { - long eventCounter = 0; - long sum = 0; - for (;;) { - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - if (eventRead.getEvent() != null) { - eventCounter++; - String[] cols = eventRead.getEvent().split(","); - long intData = Long.parseLong(cols[3]); - sum += intData; - log.info("eventCounter={}, sum={}, event={}", - String.format("%06d", eventCounter), - String.format("%08d", sum), - eventRead.getEvent()); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new UTF8StringSerializer(), + ReaderConfig.builder().build())) { + long eventCounter = 0; + long sum = 0; + for (;;) { + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + if (eventRead.getEvent() != null) { + eventCounter++; + String[] cols = eventRead.getEvent().split(","); + long intData = Long.parseLong(cols[3]); + sum += intData; + log.info("eventCounter={}, sum={}, event={}", + String.format("%06d", eventCounter), + String.format("%08d", sum), + eventRead.getEvent()); + } } } + finally { + readerGroupManager.deleteReaderGroup(readerGroup); + } } } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index cb1257be..d88dcd80 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -88,7 +88,6 @@ protected void doStart() { protected void doStop() { task.cancel(false); membershipSynchronizer.stopAsync(); - // TODO: Can we safely delete the membershipSynchronizer stream? } @Override From 9daa88355b895e098b52c2a42476b2b6af406643 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 04:58:25 +0000 Subject: [PATCH 17/78] Add graceful shutdown Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 5 ++++ .../streamprocessing/AtLeastOnceApp.java | 15 +++++++----- .../AtLeastOnceProcessor.java | 24 +++++++++++-------- 3 files changed, 28 insertions(+), 16 deletions(-) diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 6fafd4c3..1ef13f8c 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -212,6 +212,11 @@ task startExactlyOnceMultithreadedProcessor(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath } +task startAtLeastOnceApp(type: JavaExec) { + main = "io.pravega.example.streamprocessing.AtLeastOnceApp" + classpath = sourceSets.main.runtimeClasspath +} + task scriptEventDebugSink(type: CreateStartScripts) { outputDir = file('build/scripts') mainClassName = 'io.pravega.example.streamprocessing.EventDebugSink' diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 924c87e5..97750bd3 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -88,6 +88,7 @@ public void run() throws Exception { getConfig().getStream2Name(), new UTF8StringSerializer(), EventWriterConfig.builder().build())) { + final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( readerGroup, getConfig().getMembershipSynchronizerStreamName(), @@ -98,21 +99,23 @@ public void run() throws Exception { Executors.newScheduledThreadPool(1), getConfig().getHeartbeatIntervalMillis(), 1000) { - /** - * Write - * @param eventRead - */ @Override public void process(EventRead eventRead) { writer.writeEvent("0", "processed," + eventRead.getEvent()); } @Override - public void flush(EventRead eventRead) { + public void flush() { writer.flush(); } }; - processor.call(); + + processor.startAsync(); + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + System.out.println("Running shutdown hook."); + processor.stopAsync(); + })); + processor.awaitTerminated(); } } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index e4b54cb5..c6673839 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.common.util.concurrent.AbstractExecutionThreadService; import io.pravega.client.EventStreamClientFactory; import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.stream.EventRead; @@ -23,7 +24,6 @@ import org.slf4j.LoggerFactory; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; /** @@ -34,7 +34,7 @@ * each instance will get a distinct subset of events. * Instances can be in different processes. */ -abstract public class AtLeastOnceProcessor implements Callable { +abstract public class AtLeastOnceProcessor extends AbstractExecutionThreadService { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); private final ReaderGroup readerGroup; @@ -76,7 +76,7 @@ public AtLeastOnceProcessor( * has read and processed events up to the previous {@link Position}. */ @Override - public Void call() throws Exception { + protected void run() throws Exception { final String readerId = UUID.randomUUID().toString(); try (final ReaderGroupPruner pruner = ReaderGroupPruner.create( readerGroup, @@ -90,17 +90,24 @@ public Void call() throws Exception { readerGroup.getGroupName(), serializer, readerConfig)) { - for (; ; ) { + while (isRunning()) { final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); log.info("call: eventRead={}", eventRead); if (eventRead.isCheckpoint()) { - flush(eventRead); + flush(); } else if (eventRead.getEvent() != null) { process(eventRead); } } + // Gracefully stop. + // Call readNextEvent to indicate that the previous event was processed. + // When the reader is closed, it will call readerOffline with the proper position. + System.out.println("AtLeastOnceProcessor: Stopping"); + reader.readNextEvent(0); + flush(); } } + System.out.println("AtLeastOnceProcessor: Stopped"); } /** @@ -112,12 +119,9 @@ public Void call() throws Exception { abstract public void process(EventRead eventRead); /** - * This will be called when a checkpoint event is received. - * If {@link #process} did not completely process prior events, it must do so now. + * If {@link #process} did not completely process prior events, it must do so before returning. * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. - * - * @param eventRead Identifies the checkpoint name. This can generally be ignored. */ - public void flush(EventRead eventRead) { + public void flush() { } } From f17131b713f34df04e4d12b472219a94009c5fc8 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 05:42:29 +0000 Subject: [PATCH 18/78] Add logging Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 9 +++++++++ pravega-client-examples/run-from-distribution.sh | 7 +++++++ .../pravega/example/streamprocessing/AtLeastOnceApp.java | 5 ++++- .../example/streamprocessing/AtLeastOnceProcessor.java | 4 ++-- 4 files changed, 22 insertions(+), 3 deletions(-) create mode 100755 pravega-client-examples/run-from-distribution.sh diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 1ef13f8c..0cb6ccde 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -212,6 +212,14 @@ task startExactlyOnceMultithreadedProcessor(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath } +task scriptAtLeastOnceApp(type: CreateStartScripts) { + outputDir = file('build/scripts') + mainClassName = 'io.pravega.example.streamprocessing.AtLeastOnceApp' + applicationName = 'atLeastOnceApp' + defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] + classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath +} + task startAtLeastOnceApp(type: JavaExec) { main = "io.pravega.example.streamprocessing.AtLeastOnceApp" classpath = sourceSets.main.runtimeClasspath @@ -245,6 +253,7 @@ distributions { from project.scriptSecureWriter from project.scriptSecureReader from project.scriptSecureBatchReader + from project.scriptAtLeastOnceApp } into('lib') { from(jar) diff --git a/pravega-client-examples/run-from-distribution.sh b/pravega-client-examples/run-from-distribution.sh new file mode 100755 index 00000000..c283b03b --- /dev/null +++ b/pravega-client-examples/run-from-distribution.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash +set -ex +./gradlew pravega-client-examples:build +cd pravega-client-examples/build/distributions +tar -xf pravega-client-examples-0.7.0.tar +cd pravega-client-examples-0.7.0 +$* diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 97750bd3..688dcc1d 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -112,8 +112,11 @@ public void flush() { processor.startAsync(); Runtime.getRuntime().addShutdownHook(new Thread(() -> { - System.out.println("Running shutdown hook."); + log.info("Running shutdown hook."); processor.stopAsync(); + log.info("Waiting for processor to terminate."); + processor.awaitTerminated(); + log.info("Processor terminated."); })); processor.awaitTerminated(); } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index c6673839..46f23b3c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -102,12 +102,12 @@ protected void run() throws Exception { // Gracefully stop. // Call readNextEvent to indicate that the previous event was processed. // When the reader is closed, it will call readerOffline with the proper position. - System.out.println("AtLeastOnceProcessor: Stopping"); + log.info("Stopping"); reader.readNextEvent(0); flush(); } } - System.out.println("AtLeastOnceProcessor: Stopped"); + log.info("Stopped"); } /** From d2f3b26ab04f7528af91bcba8ed6e7e541de633a Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 17 Jun 2020 14:56:29 +0000 Subject: [PATCH 19/78] Fix typo in comment Signed-off-by: Claudio Fahey --- .../io/pravega/example/streamprocessing/ReaderGroupPruner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index d88dcd80..4c1cedcc 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -77,7 +77,7 @@ public void run() { @Override protected void doStart() { - // MWe must ensure that we add this reader to the membership synchronizer before the reader group. + // We must ensure that we add this reader to the membership synchronizer before the reader group. membershipSynchronizer.startAsync(); membershipSynchronizer.awaitRunning(); task = executor.scheduleAtFixedRate(new PruneRunner(), heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); From 39a9f924ae618f21c831402bbaa1c2cc603c1966 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 18 Jun 2020 04:45:48 +0000 Subject: [PATCH 20/78] Create runAtLeastOnceApp.sh Signed-off-by: Claudio Fahey --- .../{run-from-distribution.sh => runAtLeastOnceApp.sh} | 7 ++++++- .../pravega/example/streamprocessing/AppConfiguration.java | 6 +++--- 2 files changed, 9 insertions(+), 4 deletions(-) rename pravega-client-examples/{run-from-distribution.sh => runAtLeastOnceApp.sh} (61%) diff --git a/pravega-client-examples/run-from-distribution.sh b/pravega-client-examples/runAtLeastOnceApp.sh similarity index 61% rename from pravega-client-examples/run-from-distribution.sh rename to pravega-client-examples/runAtLeastOnceApp.sh index c283b03b..94061323 100755 --- a/pravega-client-examples/run-from-distribution.sh +++ b/pravega-client-examples/runAtLeastOnceApp.sh @@ -4,4 +4,9 @@ set -ex cd pravega-client-examples/build/distributions tar -xf pravega-client-examples-0.7.0.tar cd pravega-client-examples-0.7.0 -$* + +NUM_INSTANCES=${1:-1} + +for i in $(seq 1 $NUM_INSTANCES); do + bin/atLeastOnceApp >& /tmp/atLeastOnceApp-$i.log & +done diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java index e263647c..b1765e67 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java @@ -30,7 +30,7 @@ public String getScope() { } public String getReaderGroup() { - return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing-rg1"); + return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing1a-rg1"); } public String getMembershipSynchronizerStreamName() { @@ -38,7 +38,7 @@ public String getMembershipSynchronizerStreamName() { } public String getStream1Name() { - return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1"); + return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1a"); } public String getStream2Name() { @@ -54,7 +54,7 @@ public int getScaleFactor() { } public int getMinNumSegments() { - return Integer.parseInt(getEnvVar("PRAVEGA_MIN_NUM_SEGMENTS", "3")); + return Integer.parseInt(getEnvVar("PRAVEGA_MIN_NUM_SEGMENTS", "6")); } public long getCheckpointPeriodMs() { From 2af2fb7222f3cd6e3c5392b8f76cbd29aa4d36cb Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 18 Jun 2020 06:20:27 +0000 Subject: [PATCH 21/78] Now using SampleEvent class with JSON serializer. InstanceId can now be set as an application parameter. Signed-off-by: Claudio Fahey --- pravega-client-examples/runAtLeastOnceApp.sh | 4 +- .../streamprocessing/AppConfiguration.java | 21 ++++--- .../streamprocessing/AtLeastOnceApp.java | 21 ++++--- .../AtLeastOnceProcessor.java | 23 ++++---- .../streamprocessing/EventDebugSink.java | 15 +++-- .../streamprocessing/EventGenerator.java | 35 +++++------- .../streamprocessing/JSONSerializer.java | 56 +++++++++++++++++++ .../MembershipSynchronizer.java | 3 + .../example/streamprocessing/SampleEvent.java | 25 +++++++++ 9 files changed, 147 insertions(+), 56 deletions(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/JSONSerializer.java create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java diff --git a/pravega-client-examples/runAtLeastOnceApp.sh b/pravega-client-examples/runAtLeastOnceApp.sh index 94061323..1e83e16a 100755 --- a/pravega-client-examples/runAtLeastOnceApp.sh +++ b/pravega-client-examples/runAtLeastOnceApp.sh @@ -7,6 +7,6 @@ cd pravega-client-examples-0.7.0 NUM_INSTANCES=${1:-1} -for i in $(seq 1 $NUM_INSTANCES); do - bin/atLeastOnceApp >& /tmp/atLeastOnceApp-$i.log & +for i in $(seq -w 01 $NUM_INSTANCES); do + INSTANCE_ID=$i bin/atLeastOnceApp >& /tmp/atLeastOnceApp-$i.log & done diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java index b1765e67..9b651b03 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java @@ -11,6 +11,7 @@ package io.pravega.example.streamprocessing; import java.net.URI; +import java.util.UUID; /** * All parameters will come from environment variables. @@ -29,20 +30,24 @@ public String getScope() { return getEnvVar("PRAVEGA_SCOPE", "examples"); } - public String getReaderGroup() { - return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing1a-rg1"); - } - - public String getMembershipSynchronizerStreamName() { - return getReaderGroup() + "-membership"; + public String getInstanceId() { + return getEnvVar("INSTANCE_ID", UUID.randomUUID().toString()); } public String getStream1Name() { - return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1a"); + return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1c"); } public String getStream2Name() { - return getEnvVar("PRAVEGA_STREAM_2", "streamprocessing2"); + return getEnvVar("PRAVEGA_STREAM_2", "streamprocessing2c"); + } + + public String getReaderGroup() { + return getEnvVar("PRAVEGA_READER_GROUP", "streamprocessing1c-rg1"); + } + + public String getMembershipSynchronizerStreamName() { + return getReaderGroup() + "-membership"; } public int getTargetRateEventsPerSec() { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 688dcc1d..254ac22a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.gson.reflect.TypeToken; import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; import io.pravega.client.SynchronizerClientFactory; @@ -24,7 +25,6 @@ import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.UTF8StringSerializer; import org.slf4j.LoggerFactory; import java.util.concurrent.Executors; @@ -58,6 +58,8 @@ public AppConfiguration getConfig() { } public void run() throws Exception { + final String instanceId = getConfig().getInstanceId(); + log.info("instanceId={}", instanceId); final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); try (StreamManager streamManager = StreamManager.create(clientConfig)) { streamManager.createScope(getConfig().getScope()); @@ -84,15 +86,16 @@ public void run() throws Exception { final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamWriter writer = eventStreamClientFactory.createEventWriter( + EventStreamWriter writer = eventStreamClientFactory.createEventWriter( getConfig().getStream2Name(), - new UTF8StringSerializer(), + new JSONSerializer<>(new TypeToken(){}.getType()), EventWriterConfig.builder().build())) { - final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + instanceId, readerGroup, getConfig().getMembershipSynchronizerStreamName(), - new UTF8StringSerializer(), + new JSONSerializer<>(new TypeToken(){}.getType()), ReaderConfig.builder().build(), eventStreamClientFactory, synchronizerClientFactory, @@ -100,8 +103,12 @@ public void run() throws Exception { getConfig().getHeartbeatIntervalMillis(), 1000) { @Override - public void process(EventRead eventRead) { - writer.writeEvent("0", "processed," + eventRead.getEvent()); + public void process(EventRead eventRead) { + final SampleEvent event = eventRead.getEvent(); + event.processedBy = instanceId; + event.processedLatencyMs = System.currentTimeMillis() - event.timestamp; + log.info("{}", event); + writer.writeEvent(event.routingKey, event); } @Override diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 46f23b3c..9ebc3756 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.UUID; import java.util.concurrent.ScheduledExecutorService; /** @@ -34,12 +33,13 @@ * each instance will get a distinct subset of events. * Instances can be in different processes. */ -abstract public class AtLeastOnceProcessor extends AbstractExecutionThreadService { +abstract public class AtLeastOnceProcessor extends AbstractExecutionThreadService { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); + private final String instanceId; private final ReaderGroup readerGroup; private final String membershipSynchronizerStreamName; - private final Serializer serializer; + private final Serializer serializer; private final ReaderConfig readerConfig; private final EventStreamClientFactory eventStreamClientFactory; private final SynchronizerClientFactory synchronizerClientFactory; @@ -48,15 +48,17 @@ abstract public class AtLeastOnceProcessor extends AbstractExecutionThreadServic private final long readTimeoutMillis; public AtLeastOnceProcessor( + String instanceId, ReaderGroup readerGroup, String membershipSynchronizerStreamName, - Serializer serializer, + Serializer serializer, ReaderConfig readerConfig, EventStreamClientFactory eventStreamClientFactory, SynchronizerClientFactory synchronizerClientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis, long readTimeoutMillis) { + this.instanceId = instanceId; this.readerGroup = readerGroup; this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; this.serializer = serializer; @@ -77,22 +79,21 @@ public AtLeastOnceProcessor( */ @Override protected void run() throws Exception { - final String readerId = UUID.randomUUID().toString(); try (final ReaderGroupPruner pruner = ReaderGroupPruner.create( readerGroup, membershipSynchronizerStreamName, - readerId, + instanceId, synchronizerClientFactory, executor, heartbeatIntervalMillis)) { - try (final EventStreamReader reader = eventStreamClientFactory.createReader( - readerId, + try (final EventStreamReader reader = eventStreamClientFactory.createReader( + instanceId, readerGroup.getGroupName(), serializer, readerConfig)) { while (isRunning()) { - final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); - log.info("call: eventRead={}", eventRead); + final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); + log.info("eventRead={}", eventRead); if (eventRead.isCheckpoint()) { flush(); } else if (eventRead.getEvent() != null) { @@ -116,7 +117,7 @@ protected void run() throws Exception { * * @param eventRead The event read. */ - abstract public void process(EventRead eventRead); + abstract public void process(EventRead eventRead); /** * If {@link #process} did not completely process prior events, it must do so before returning. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 55ec8b1b..18ed6ac4 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.gson.reflect.TypeToken; import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.ReaderGroupManager; @@ -70,23 +71,21 @@ public void run() throws Exception { try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI())) { readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamReader reader = clientFactory.createReader( + EventStreamReader reader = clientFactory.createReader( "reader", readerGroup, - new UTF8StringSerializer(), + new JSONSerializer<>(new TypeToken(){}.getType()), ReaderConfig.builder().build())) { long eventCounter = 0; long sum = 0; for (;;) { - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); if (eventRead.getEvent() != null) { eventCounter++; - String[] cols = eventRead.getEvent().split(","); - long intData = Long.parseLong(cols[3]); - sum += intData; + sum += eventRead.getEvent().intData; log.info("eventCounter={}, sum={}, event={}", - String.format("%06d", eventCounter), - String.format("%08d", sum), + String.format("%6d", eventCounter), + String.format("%8d", sum), eventRead.getEvent()); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 573b8c0c..b72d7920 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.gson.reflect.TypeToken; import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.StreamManager; @@ -61,31 +62,25 @@ public void run() throws Exception { Random rand = new Random(42); try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamWriter writer = clientFactory.createEventWriter( + EventStreamWriter writer = clientFactory.createEventWriter( getConfig().getStream1Name(), - new UTF8StringSerializer(), + new JSONSerializer<>(new TypeToken(){}.getType()), EventWriterConfig.builder().build())) { - long eventCounter = 0; + long sequenceNumber = 0; long sum = 0; final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); for (;;) { - eventCounter++; - long intData = rand.nextInt(100); - String routingKey = String.format("%02d", intData); - sum += intData; - String generatedTimestampStr = dateFormat.format(new Date()); - String message = String.join(",", - String.format("%06d", eventCounter), - routingKey, - String.format("%02d", intData), - String.format("%08d", sum), - generatedTimestampStr - ); - log.info("eventCounter={}, sum={}, event={}", - String.format("%06d", eventCounter), - String.format("%08d", sum), - message); - final CompletableFuture writeFuture = writer.writeEvent(routingKey, message); + sequenceNumber++; + final SampleEvent event = new SampleEvent(); + event.sequenceNumber = sequenceNumber; + event.routingKey = String.format("%3d", rand.nextInt(1000)); + event.intData = rand.nextInt(1000); + sum += event.intData; + event.sum = sum; + event.timestamp = System.currentTimeMillis(); + event.timestampStr = dateFormat.format(new Date(event.timestamp)); + log.info("{}", event); + final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); Thread.sleep(1000); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/JSONSerializer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/JSONSerializer.java new file mode 100644 index 00000000..7cbdead9 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/JSONSerializer.java @@ -0,0 +1,56 @@ +/** + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + */ +package io.pravega.example.streamprocessing; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import io.pravega.client.stream.Serializer; +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +import java.lang.reflect.Type; +import java.nio.ByteBuffer; + +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * An implmentation of {@link Serializer} that uses JSON serialization. + * @param Type of Object to be serialized. + */ +@Slf4j +@AllArgsConstructor +public class JSONSerializer implements Serializer { + + /** + * This is used to represent the generic type. The user can use {@link com.google.gson.reflect.TypeToken} to represent + * the type of the object which will be serialized/deserialized. + */ + private final Type type; + private final Gson gson = new GsonBuilder().create(); + + /** + * {@inheritDoc} + * JSON based serialization. + */ + @Override + public ByteBuffer serialize(T value) { + return ByteBuffer.wrap(gson.toJson(value, type).getBytes(UTF_8)); + } + + /** + * {@inheritDoc} + * JSON based deserialization. + */ + @Override + public T deserialize(ByteBuffer serializedValue) { + return gson.fromJson(UTF_8.decode(serializedValue).toString(), type); + } + +} \ No newline at end of file diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 56bcf9b0..d59c90b0 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -60,6 +60,9 @@ public class MembershipSynchronizer extends AbstractService { */ private static final int DEATH_THRESHOLD = 10; + /** + * Unique identifier for this member. + */ private final String instanceId; /** diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java new file mode 100644 index 00000000..53368215 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java @@ -0,0 +1,25 @@ +package io.pravega.example.streamprocessing; + +public class SampleEvent { + public long sequenceNumber; + public String routingKey; + public long intData; + public long sum; + public long timestamp; + public String timestampStr; + public long processedLatencyMs; + public String processedBy; + + @Override + public String toString() { + return "SampleEvent{" + + "sequenceNumber=" + String.format("%6d", sequenceNumber) + + ", routingKey=" + routingKey + + ", intData=" + String.format("%3d", intData) + + ", sum=" + String.format("%8d", sum) + + ", timestampStr=" + timestampStr + + ", processedLatencyMs=" + String.format("%6d", processedLatencyMs) + + ", processedBy=" + processedBy + + '}'; + } +} From c18bfe9f488cd9ff852230c15f2a7bc2ef82ac5d Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 20 Jun 2020 18:14:05 +0000 Subject: [PATCH 22/78] Update README (incomplete) Signed-off-by: Claudio Fahey --- .../example/streamprocessing/README.md | 50 ++++++++++--------- 1 file changed, 27 insertions(+), 23 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index 51056d7b..f84d4c20 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -16,17 +16,23 @@ These examples include: This application generates new events every 1 second and writes them to a Pravega stream (referred to as stream1). -- (OBSOLETE) [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java): - This application continuously reads events from stream1, performs a stateless computation - to generate output events, and writes the output event to another - Pravega stream (referred to as stream2). - It uses the exactly-once algorithms described below. +- [AtLeastOnceApp](AtLeastOnceApp.java): + This application demonstrates reading events from a Pravega stream (stream1), processing each event, + and writing each output event to another Pravega stream (stream2). + It guarantees that each event is processed at least once, using the algorithms described below. + If multiple instances of this application are executed using the same readerGroupName parameter, + each instance will get a distinct subset of events, providing load balancing and redundancy. + The user-defined processing function must be stateless. + + There is no direct dependency on other systems, except for Pravega. + In particular, this application does not *directly* use a distributed file system, nor ZooKeeper. + Coordination between processes occurs only through Pravega streams such as + Reader Groups and State Synchronizers. - [EventDebugSink](EventDebugSink.java): This application reads events from stream2 and displays them on the console. - # How to Run - Enable INFO level logging by editing the file [logback.xml](../../../../../resources/logback.xml). @@ -36,8 +42,8 @@ These examples include: ``` - Start the event generator. - ``` - cd pravega-samples + ```shell script + cd ~/pravega-samples ./gradlew pravega-client-examples:startEventGenerator ``` @@ -45,38 +51,36 @@ These examples include: You can either set them in your shell (`export PRAVEGA_SCOPE=examples`) or use the below syntax. If you are using a non-local Pravega instance, specify the controller as follows: - ``` + ```shell script PRAVEGA_CONTROLLER=tcp://pravega.example.com:9090 ./gradlew pravega-client-examples:startEventGenerator ``` Multiple parameters can be specified as follows. - ``` + ```shell script PRAVEGA_SCOPE=examples PRAVEGA_CONTROLLER=tcp://localhost:9090 ./gradlew pravega-client-examples:startEventGenerator ``` See [Parameters.java](Parameters.java) for available appConfiguration. -- In another window, start the stream processor: - ``` - ./gradlew pravega-client-examples:startExactlyOnceMultithreadedProcessor +- In another window, start one or more instances of the stream processor. + The `runAtLeastOnceApp.sh` can be used to run multiple instances concurrently. + ```shell script + cd ~/pravega-samples/pravega-client-examples + ./runAtLeastOnceApp.sh 2 ``` - In another window, start the event debug sink: - ``` + ```shell script ./gradlew pravega-client-examples:startEventDebugSink ``` -- (OBSOLETE) Note: The [ExactlyOnceMultithreadedProcessor](ExactlyOnceMultithreadedProcessor.java) - will automatically restart from the latest checkpoint. - However, if Pravega streams are truncated or deleted, or checkpoint files in - /tmp/checkpoints are deleted or otherwise bad, you may need to start over from - a clean system. To do so, follow these steps: - - Stop the event generator, stream processor, and event debug sink. - - Delete the contents of /tmp/checkpoints. - - Use a new scope (PRAVEGA_SCOPE) or streams (PRAVEGA_STREAM_1 and PRAVEGA_STREAM_2). +# (OBSOLETE) Achieving At-Least-Once Semantics +Pravega has a sophisticated concept called a +[Reader Group](http://pravega.io/docs/latest/reader-group-design/). -# (OBSOLETE) Achieving At-Least-Once Semantics +The complete state of a Reader Group is maintained by each reader. +Each reader reads updates from the Reader Group stream. The current position in a Pravega stream is defined by a stream cut. A stream cut is essentially a mapping from segment numbers to byte offsets From 353b3468a9b57aa2f9dd8e81ee2c0d641a921476 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 20 Jun 2020 20:45:11 +0000 Subject: [PATCH 23/78] Update README (incomplete) Signed-off-by: Claudio Fahey --- .../example/streamprocessing/README.md | 109 ++---------------- 1 file changed, 7 insertions(+), 102 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index f84d4c20..c36eebeb 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -3,12 +3,8 @@ # Overview The examples in this directory are intended to illustrate -how at-least-once and exactly-once semantics can be achieved with Pravega. -To make it clear to the reader, the entirety of the stream processor is defined -within this package with minimal dependencies. -In particular, these illustrative examples do *not* use Apache Flink. -In addition to exactly-once semantics, Apache Flink provides -windowing, aggregations, event time processing, and stateful processing. +how at-least-once semantics can be achieved with Pravega. +These show how to use Pravega directly, without relying on an external framework such as Apache Flink. These examples include: @@ -19,8 +15,8 @@ These examples include: - [AtLeastOnceApp](AtLeastOnceApp.java): This application demonstrates reading events from a Pravega stream (stream1), processing each event, and writing each output event to another Pravega stream (stream2). - It guarantees that each event is processed at least once, using the algorithms described below. - If multiple instances of this application are executed using the same readerGroupName parameter, + It guarantees that each event is processed at least once, even in the presence of failures. + If multiple instances of this application are executed using the same reader group name, each instance will get a distinct subset of events, providing load balancing and redundancy. The user-defined processing function must be stateless. @@ -35,7 +31,7 @@ These examples include: # How to Run -- Enable INFO level logging by editing the file [logback.xml](../../../../../resources/logback.xml). +- (Optional) Enable INFO level logging by editing the file [logback.xml](../../../../../resources/logback.xml). Update it to include: ``` @@ -74,7 +70,7 @@ These examples include: ./gradlew pravega-client-examples:startEventDebugSink ``` -# (OBSOLETE) Achieving At-Least-Once Semantics +# Achieving At-Least-Once Semantics Pravega has a sophisticated concept called a [Reader Group](http://pravega.io/docs/latest/reader-group-design/). @@ -82,95 +78,4 @@ Pravega has a sophisticated concept called a The complete state of a Reader Group is maintained by each reader. Each reader reads updates from the Reader Group stream. -The current position in a Pravega stream is defined by a stream cut. -A stream cut is essentially a mapping from segment numbers to byte offsets -within that segment. -However, because the number of concurrent segments can dynamically -increase and decrease for a single stream, it is not possible for -a Pravega application to maintain this mapping by itself. -To achieve at-least-once semantics with Pravega, use the algorithms in this section. - -In the algorithms below, one process in a distributed system will be designated -by the user as a master. There can be any number of worker processes. - -## Checkpoint Algorithm - -- Master - - Create a checkpoint name as a new random UUID. - - Create a directory in persistent storage (e.g. NFS/HDFS) to store information - about this checkpoint. - For instance: /checkpoints/5ef9b301-2af1-4320-8028-c4cef9f39aca - - Initiate a checkpoint on the reader group used by the workers. - - Wait for all workers to complete handling the checkpoint. - - After waiting, the Pravega checkpoint object, including the - stream cuts, will be available. - - Write the Pravega checkpoint object to the checkpoint directory. - - Atomically update the file /checkpoints/latest to reference the - checkpoint name. - -- Worker - - Loop: - - Read the next event from the reader group. - Immediately after a checkpoint event, reading the next event - will notify the master that it has finished handling the checkpoint. - - If this is a checkpoint event: - - Write worker state to the checkpoint directory. - - If this is a normal event: - - Process the event and write output event(s). - -## Restart/Recovery Algorithm - -- Master: - - Read the file /checkpoints/latest to determine the directory - containing the latest successful checkpoint. - - Read the Pravega checkpoint object to determine the stream cut - at the last checkpoint. - - Create a reader group that starts reading from this stream cut. - - Launch worker processes, passing the name of the checkpoint and - the reader group. - -- Worker: - - Load worker state from the checkpoint directory. - - Create a Pravega reader that reads from the reader group. - - Process events as usual. - - -# (OBSOLETE) Achieving Exactly-Once Semantics - -Exactly-once semantics can achieved by starting with at-least-once -semantics and adding write idempotence. -In Pravega, write idempotence can be achieved by using Pravega -transactions. - -Essentially, the at-least-once algorithm is modified in the following ways: - -- Each workers start its own Pravega transaction. - All output events are written to the transaction. - If a worker writes to multiple Pravega streams, it would use - a different transaction for each stream. - -- When a worker receives a checkpoint event, it flushes - (but does not commit) the transaction - and adds the transaction ID to the checkpoint directory. - -- When all workers complete handling of the checkpoint, the master - updates the checkpoint directory as before and then - commits all transactions referenced in the checkpoint directory. - -- Upon restart, any transactions referenced in the checkpoint - directory are committed if they have not yet been committed. - - Note: If any transactions fail to commit (usually due to a - transaction timeout), the stream processor will terminate - as it cannot guarantee exactly-once semantics. - To maintain correctness, recovery - from such a situation will usually require deleting the output stream - and recreating it from the beginning. - To avoid such problems, ensure that failed stream processors - are restarted before transactions timeout. - See also - [EventWriterConfig.java](https://github.com/pravega/pravega/blob/r0.4/client/src/main/java/io/pravega/client/stream/EventWriterConfig.java#L27). - - -See [ExactlyOnceMultithreadedProcessor.java](ExactlyOnceMultithreadedProcessor.java) -for details. +TODO: Finish this section. From f31a483b57215306eb1843ab70cb7ede205db49b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 20 Jun 2020 21:07:51 +0000 Subject: [PATCH 24/78] Remove unused test code. Revert changes to logback.xml. Add new scripts to build.gradle. Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 19 +- .../src/main/resources/logback.xml | 2 +- .../java/io/pravega/utils/SetupUtils.java | 361 ------------------ 3 files changed, 3 insertions(+), 379 deletions(-) delete mode 100644 pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 0cb6ccde..892c12d4 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -20,16 +20,12 @@ archivesBaseName = 'pravega-client-examples' resources ext { -// junitJupiterVersion = '5.3.1' junitVersion = '4.11' commonsCLIVersion = '1.4' } dependencies { testCompile "junit:junit:${junitVersion}" -// testCompile "org.junit.jupiter:junit-jupiter-engine:${junitJupiterVersion}" - testCompile "io.pravega:pravega-standalone:${pravegaVersion}" - compileOnly "org.projectlombok:lombok:${lombokVersion}" compile "io.pravega:pravega-client:${pravegaVersion}", "io.pravega:pravega-common:${pravegaVersion}", @@ -199,19 +195,6 @@ task startEventGenerator(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath } -task scriptExactlyOnceMultithreadedProcessor(type: CreateStartScripts) { - outputDir = file('build/scripts') - mainClassName = 'io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor' - applicationName = 'exactlyOnceMultithreadedProcessor' - defaultJvmOpts = ["-Dlogback.configurationFile=file:conf/logback.xml"] - classpath = files(jar.archivePath) + sourceSets.main.runtimeClasspath -} - -task startExactlyOnceMultithreadedProcessor(type: JavaExec) { - main = "io.pravega.example.streamprocessing.ExactlyOnceMultithreadedProcessor" - classpath = sourceSets.main.runtimeClasspath -} - task scriptAtLeastOnceApp(type: CreateStartScripts) { outputDir = file('build/scripts') mainClassName = 'io.pravega.example.streamprocessing.AtLeastOnceApp' @@ -253,7 +236,9 @@ distributions { from project.scriptSecureWriter from project.scriptSecureReader from project.scriptSecureBatchReader + from project.scriptEventGenerator from project.scriptAtLeastOnceApp + from project.scriptEventDebugSink } into('lib') { from(jar) diff --git a/pravega-client-examples/src/main/resources/logback.xml b/pravega-client-examples/src/main/resources/logback.xml index 34f9c7c0..ae1a2a69 100644 --- a/pravega-client-examples/src/main/resources/logback.xml +++ b/pravega-client-examples/src/main/resources/logback.xml @@ -18,7 +18,7 @@ - + diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java deleted file mode 100644 index dc92bff6..00000000 --- a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java +++ /dev/null @@ -1,361 +0,0 @@ -/* - * Copyright 2019 Flavio Junqueira - * - * Licensed 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 io.pravega.utils; - -import com.google.common.base.Preconditions; -import io.pravega.client.ClientConfig; -import io.pravega.client.EventStreamClientFactory; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.ScalingPolicy; -import io.pravega.client.stream.Stream; -import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.Controller; -import io.pravega.client.stream.impl.ControllerImpl; -import io.pravega.client.stream.impl.ControllerImplConfig; -import io.pravega.common.concurrent.ExecutorServiceHelpers; -import io.pravega.local.InProcPravegaCluster; -import lombok.Cleanup; -import lombok.Getter; -import lombok.Setter; -import org.apache.commons.lang3.RandomStringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.concurrent.NotThreadSafe; -import java.io.IOException; -import java.io.InputStream; -import java.net.ServerSocket; -import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardCopyOption; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Utility functions for creating the test setup. - */ -//@Slf4j -@NotThreadSafe -public final class SetupUtils { - - static Logger log = LoggerFactory.getLogger(SetupUtils.class); - private static final ScheduledExecutorService DEFAULT_SCHEDULED_EXECUTOR_SERVICE = ExecutorServiceHelpers.newScheduledThreadPool(3, "SetupUtils"); - - private final PravegaGateway gateway; - - // Manage the state of the class. - private final AtomicBoolean started = new AtomicBoolean(false); - - // auth enabled by default. Set it to false to disable Pravega authentication and authorization. - @Setter - private boolean enableAuth = false; - - // Set to true to enable TLS - @Setter - private boolean enableTls = false; - - @Setter - private boolean enableHostNameValidation = false; - - private boolean enableRestServer = true; - - // The test Scope name. - @Getter - private final String scope = RandomStringUtils.randomAlphabetic(20); - - public SetupUtils() { - this(System.getProperty("pravega.uri")); - } - - public SetupUtils(String externalUri) { - log.info("SetupUtils constructor"); - if (externalUri != null) { - log.info("Using Pravega services at {}.", externalUri); - gateway = new ExternalPravegaGateway(URI.create(externalUri)); - } else { - log.info("Starting in-process Pravega services."); - gateway = new InProcPravegaGateway(); - } - log.info("Done with constructor {}", gateway.toString()); - } - - - /** - * Start all pravega related services required for the test deployment. - * - * @throws Exception on any errors. - */ - public void startAllServices() throws Exception { - log.info("Starting all services"); - if (!this.started.compareAndSet(false, true)) { - log.warn("Services already started, not attempting to start again"); - return; - } - log.info("Starting gateway"); - gateway.start(); - } - - /** - * Stop the pravega cluster and release all resources. - * - * @throws Exception on any errors. - */ - public void stopAllServices() throws Exception { - if (!this.started.compareAndSet(true, false)) { - log.warn("Services not yet started or already stopped, not attempting to stop"); - return; - } - - try { - gateway.stop(); - } catch (Exception e) { - log.warn("Services did not stop cleanly (" + e.getMessage() + ")", e); - } - } - - /** - * Get resources as temp file. - * - * @param resourceName Name of the resource. - * - * @return Path of the temp file. - */ - static String getFileFromResource(String resourceName) { - try { - Path tempPath = Files.createTempFile("test-", ".tmp"); - tempPath.toFile().deleteOnExit(); - try (InputStream stream = SetupUtils.class.getClassLoader().getResourceAsStream(resourceName)) { - Files.copy(SetupUtils.class.getClassLoader().getResourceAsStream(resourceName), tempPath, StandardCopyOption.REPLACE_EXISTING); - } - return tempPath.toFile().getAbsolutePath(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - - /** - * Fetch the controller endpoint for this cluster. - * - * @return URI The controller endpoint to connect to this cluster. - */ - public URI getControllerUri() { - return getClientConfig().getControllerURI(); - } - - /** - * Fetch the client configuration with which to connect to the controller. - */ - public ClientConfig getClientConfig() { - return this.gateway.getClientConfig(); - } - - /** - * Create a controller facade for this cluster. - * @return The controller facade, which must be closed by the caller. - */ - public Controller newController() { - ControllerImplConfig config = ControllerImplConfig.builder() - .clientConfig(getClientConfig()) - .build(); - return new ControllerImpl(config, DEFAULT_SCHEDULED_EXECUTOR_SERVICE); - } - - /** - * Create a {@link EventStreamClientFactory} for this cluster and scope. - */ - public EventStreamClientFactory newClientFactory() { - return EventStreamClientFactory.withScope(this.scope, getClientConfig()); - } - - /** - * Create the test stream. - * - * @param streamName Name of the test stream. - * @param numSegments Number of segments to be created for this stream. - * - * @throws Exception on any errors. - */ - public void createTestStream(final String streamName, final int numSegments) - throws Exception { - Preconditions.checkState(this.started.get(), "Services not yet started"); - Preconditions.checkNotNull(streamName); - Preconditions.checkArgument(numSegments > 0); - - @Cleanup - StreamManager streamManager = StreamManager.create(getClientConfig()); - streamManager.createScope(this.scope); - streamManager.createStream(this.scope, streamName, - StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.fixed(numSegments)) - .build()); - log.info("Created stream: " + streamName); - } - - /** - * Get the stream. - * - * @param streamName Name of the test stream. - * - * @return a Stream - */ - public Stream getStream(final String streamName) { - return Stream.of(this.scope, streamName); - } - - /** - * A gateway interface to Pravega for integration test purposes. - */ - private interface PravegaGateway { - /** - * Starts the gateway. - */ - void start() throws Exception; - - /** - * Stops the gateway. - */ - void stop() throws Exception; - - /** - * Gets the client configuration with which to connect to the controller. - */ - ClientConfig getClientConfig(); - } - - class InProcPravegaGateway implements PravegaGateway { - - // The pravega cluster. - private InProcPravegaCluster inProcPravegaCluster = null; - - @Override - public void start() throws Exception { - log.info("Starting gateway"); - int zkPort = PortUtils.getAvailableListenPort(); - int controllerPort = PortUtils.getAvailableListenPort(); - int hostPort = PortUtils.getAvailableListenPort(); - int restPort = PortUtils.getAvailableListenPort(); - - log.info("Building"); - this.inProcPravegaCluster = InProcPravegaCluster.builder() - .isInProcZK(true) - .secureZK(enableTls) //configure ZK for security - .zkUrl("localhost:" + zkPort) - .zkPort(zkPort) - .isInMemStorage(true) - .isInProcController(true) - .controllerCount(1) - .restServerPort(restPort) - .enableRestServer(enableRestServer) - .isInProcSegmentStore(true) - .segmentStoreCount(1) - .containerCount(4) - .enableMetrics(false) - .enableAuth(enableAuth) - .enableTls(enableTls) - //.certFile(getFileFromResource(CERT_FILE)) // pravega #2519 - //.keyFile(getFileFromResource(KEY_FILE)) - //.jksKeyFile(getFileFromResource(STANDALONE_KEYSTORE_FILE)) - //.jksTrustFile(getFileFromResource(STANDALONE_TRUSTSTORE_FILE)) - //.keyPasswordFile(getFileFromResource(STANDALONE_KEYSTORE_PASSWD_FILE)) - //.passwdFile(getFileFromResource(PASSWD_FILE)) - //.userName(PRAVEGA_USERNAME) - //.passwd(PRAVEGA_PASSWORD) - .build(); - log.info("Done building"); - this.inProcPravegaCluster.setControllerPorts(new int[]{controllerPort}); - this.inProcPravegaCluster.setSegmentStorePorts(new int[]{hostPort}); - this.inProcPravegaCluster.start(); - log.info("Initialized Pravega Cluster"); - log.info("Controller port is {}", controllerPort); - log.info("Host port is {}", hostPort); - log.info("REST server port is {}", restPort); - } - - @Override - public void stop() throws Exception { - inProcPravegaCluster.close(); - } - - @Override - public ClientConfig getClientConfig() { - log.info("Getting client config"); - return ClientConfig.builder() - .controllerURI(URI.create(inProcPravegaCluster.getControllerURI())) - //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) - //.validateHostName(enableHostNameValidation) - //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) - .build(); - } - } - - class ExternalPravegaGateway implements PravegaGateway { - - private final URI controllerUri; - - public ExternalPravegaGateway(URI controllerUri) { - this.controllerUri = controllerUri; - } - - @Override - public void start() throws Exception { - } - - @Override - public void stop() throws Exception { - } - - @Override - public ClientConfig getClientConfig() { - return ClientConfig.builder() - .controllerURI(controllerUri) - //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) - //.validateHostName(enableHostNameValidation) - //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) - .build(); - } - } - - static class PortUtils { - // Linux uses ports from range 32768 - 61000. - private static final int BASE_PORT = 32768; - private static final int MAX_PORT_COUNT = 28232; - private static final AtomicInteger NEXT_PORT = new AtomicInteger(1); - - /** - * A helper method to get a random free port. - * - * @return free port. - */ - public static int getAvailableListenPort() { - for (int i = 0; i < MAX_PORT_COUNT; i++) { - int candidatePort = BASE_PORT + NEXT_PORT.getAndIncrement() % MAX_PORT_COUNT; - try { - ServerSocket serverSocket = new ServerSocket(candidatePort); - serverSocket.close(); - return candidatePort; - } catch (IOException e) { - // Do nothing. Try another port. - } - } - throw new IllegalStateException( - String.format("Could not assign port in range %d - %d", BASE_PORT, MAX_PORT_COUNT + BASE_PORT)); - } - } -} From 753c7f6871894646970075361337d7139315ba77 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:36:27 +0000 Subject: [PATCH 25/78] Remove unused test class. Signed-off-by: Claudio Fahey --- .../pravega/example/StreamProcessingTest.java | 99 ------------------- 1 file changed, 99 deletions(-) delete mode 100644 pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java diff --git a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java deleted file mode 100644 index 947592d9..00000000 --- a/pravega-client-examples/src/test/java/io/pravega/example/StreamProcessingTest.java +++ /dev/null @@ -1,99 +0,0 @@ -package io.pravega.example; - -import io.pravega.client.ClientConfig; -import io.pravega.client.EventStreamClientFactory; -import io.pravega.client.admin.ReaderGroupManager; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.EventRead; -import io.pravega.client.stream.EventStreamReader; -import io.pravega.client.stream.EventStreamWriter; -import io.pravega.client.stream.EventWriterConfig; -import io.pravega.client.stream.ReaderConfig; -import io.pravega.client.stream.ReaderGroupConfig; -import io.pravega.client.stream.ScalingPolicy; -import io.pravega.client.stream.Serializer; -import io.pravega.client.stream.Stream; -import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.UTF8StringSerializer; -import io.pravega.example.streamprocessing.AtLeastOnceProcessor; -import io.pravega.utils.SetupUtils; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; - -public class StreamProcessingTest { - static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); - - protected static final AtomicReference SETUP_UTILS = new AtomicReference<>(); - - @BeforeClass - public static void setup() throws Exception { - SETUP_UTILS.set(new SetupUtils()); - SETUP_UTILS.get().startAllServices(); - } - - @AfterClass - public static void tearDown() throws Exception { - SETUP_UTILS.get().stopAllServices(); - } - - public static void runWorker(final ClientConfig clientConfig, - final String inputStreamName, - final String readerGroup) throws Exception { - - } - - @Test - public void basicTest() throws Exception { - final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); - log.info("Test case: {}", methodName); - - final String inputStreamName = "stream-" + UUID.randomUUID().toString(); - final String readerGroup = "rg" + UUID.randomUUID().toString().replace("-", ""); - final String scope = SETUP_UTILS.get().getScope(); - final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); - final String readerId = "reader-" + UUID.randomUUID().toString(); - final ReaderConfig readerConfig = ReaderConfig.builder().build(); - final Serializer serializer = new UTF8StringSerializer(); - final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() - .stream(Stream.of(scope, inputStreamName)) - .build(); - final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); - -// try (StreamManager streamManager = StreamManager.create(clientConfig)) { -// streamManager.createScope(scope); -// -// StreamConfiguration streamConfig = StreamConfiguration.builder() -// .scalingPolicy(ScalingPolicy.fixed(3)) -// .build(); -// streamManager.createStream(scope, inputStreamName, streamConfig); -// -// try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { -// readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); -// } -// try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); -// final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { -// -// inputWriter.writeEvent("CLAUDIO1"); -// -// AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { -// @Override -// public EventStreamReader createReader() { -// return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); -// } -// -// @Override -// public void write(EventRead eventRead) { -// } -// }; -// processor.call(); -// // TODO: Wait for correct result and then terminate. -// } -// } - } -} From faddc1471529cfbcc3116c44ab44bca0b35f2a1a Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:40:00 +0000 Subject: [PATCH 26/78] EventGenerator logs when events are acked. Signed-off-by: Claudio Fahey --- .../pravega/example/streamprocessing/EventGenerator.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index b72d7920..469a72f2 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -18,7 +18,7 @@ import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.text.SimpleDateFormat; @@ -30,12 +30,12 @@ * A simple example app to write messages to a Pravega stream. */ public class EventGenerator { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventGenerator.class); + private static final Logger log = LoggerFactory.getLogger(EventGenerator.class); private final AppConfiguration config; public static void main(String[] args) throws Exception { - EventGenerator app = new EventGenerator(new AppConfiguration(args)); + final EventGenerator app = new EventGenerator(new AppConfiguration(args)); app.run(); } @@ -58,6 +58,7 @@ public void run() throws Exception { getConfig().getMinNumSegments())) .build(); streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); + streamManager.updateStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); } Random rand = new Random(42); @@ -81,6 +82,8 @@ public void run() throws Exception { event.timestampStr = dateFormat.format(new Date(event.timestamp)); log.info("{}", event); final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); + final long ackedSequenceNumber = sequenceNumber; + writeFuture.thenRun(() -> log.debug("Acknowledged: sequenceNumber={}", ackedSequenceNumber)); Thread.sleep(1000); } } From 0d6ebd13ef34e548c34d105993185c9af1e88f14 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:42:28 +0000 Subject: [PATCH 27/78] Minor logging and formatting changes. Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 33 ++++++++++--------- .../MembershipSynchronizer.java | 4 +-- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 9ebc3756..706bc8c9 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -85,28 +85,28 @@ protected void run() throws Exception { instanceId, synchronizerClientFactory, executor, - heartbeatIntervalMillis)) { - try (final EventStreamReader reader = eventStreamClientFactory.createReader( + heartbeatIntervalMillis); + final EventStreamReader reader = eventStreamClientFactory.createReader( instanceId, readerGroup.getGroupName(), serializer, readerConfig)) { - while (isRunning()) { - final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); - log.info("eventRead={}", eventRead); - if (eventRead.isCheckpoint()) { - flush(); - } else if (eventRead.getEvent() != null) { - process(eventRead); - } + + while (isRunning()) { + final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); + log.info("eventRead={}", eventRead); + if (eventRead.isCheckpoint()) { + flush(); + } else if (eventRead.getEvent() != null) { + process(eventRead); } - // Gracefully stop. - // Call readNextEvent to indicate that the previous event was processed. - // When the reader is closed, it will call readerOffline with the proper position. - log.info("Stopping"); - reader.readNextEvent(0); - flush(); } + // Gracefully stop. + // Call readNextEvent to indicate that the previous event was processed. + // When the reader is closed, it will call readerOffline with the proper position. + log.info("Stopping"); + reader.readNextEvent(0); + flush(); } log.info("Stopped"); } @@ -114,6 +114,7 @@ protected void run() throws Exception { /** * Process an event that was read. * Processing can be performed asynchronously after this method returns. + * This method must be stateless. * * @param eventRead The event read. */ diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index d59c90b0..8699a751 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -159,7 +159,7 @@ public void run() { try { stateSync.fetchUpdates(); notifyListener(); - log.info("run: BEGIN: vectorTime={}, isOverUnconditionalThreshold={}, liveInstances={}", + log.debug("run: BEGIN: vectorTime={}, isOverUnconditionalThreshold={}, liveInstances={}", stateSync.getState().getVectorTime(), stateSync.getState().isOverUnconditionalThreshold(instanceId), stateSync.getState().liveInstances); @@ -178,7 +178,7 @@ public void run() { stateSync.fetchUpdates(); } notifyListener(); - log.info("run: END: vectorTime={}, liveInstances={}", + log.debug("run: END: vectorTime={}, liveInstances={}", stateSync.getState().getVectorTime(), stateSync.getState().liveInstances); } catch (Exception e) { From a1f3a5ff86c4487557c7d502c900d36d00000346 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:44:32 +0000 Subject: [PATCH 28/78] Move runAtLeastOnceApp.sh Signed-off-by: Claudio Fahey --- {pravega-client-examples => scripts}/runAtLeastOnceApp.sh | 2 ++ 1 file changed, 2 insertions(+) rename {pravega-client-examples => scripts}/runAtLeastOnceApp.sh (88%) diff --git a/pravega-client-examples/runAtLeastOnceApp.sh b/scripts/runAtLeastOnceApp.sh similarity index 88% rename from pravega-client-examples/runAtLeastOnceApp.sh rename to scripts/runAtLeastOnceApp.sh index 1e83e16a..10979900 100755 --- a/pravega-client-examples/runAtLeastOnceApp.sh +++ b/scripts/runAtLeastOnceApp.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash set -ex +ROOT_DIR=$(dirname $0)/.. +cd ${ROOT_DIR} ./gradlew pravega-client-examples:build cd pravega-client-examples/build/distributions tar -xf pravega-client-examples-0.7.0.tar From 82a074c6f2426231ee697f7588707f329452ff60 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:46:55 +0000 Subject: [PATCH 29/78] Documentation, logging, and formatting updates Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 32 ++++- .../streamprocessing/EventDebugSink.java | 4 +- .../example/streamprocessing/README.md | 124 +++++++++++++++--- .../streamprocessing/ReaderGroupPruner.java | 11 +- .../example/streamprocessing/SampleEvent.java | 10 ++ 5 files changed, 156 insertions(+), 25 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 254ac22a..15867c3c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -25,6 +25,7 @@ import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; +import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.Executors; @@ -40,12 +41,12 @@ * to view the output events. */ public class AtLeastOnceApp { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(AtLeastOnceApp.class); + private static final Logger log = LoggerFactory.getLogger(AtLeastOnceApp.class); private final AppConfiguration config; public static void main(String[] args) throws Exception { - AtLeastOnceApp app = new AtLeastOnceApp(new AppConfiguration(args)); + final AtLeastOnceApp app = new AtLeastOnceApp(new AppConfiguration(args)); app.run(); } @@ -58,9 +59,15 @@ public AppConfiguration getConfig() { } public void run() throws Exception { + // Get the provided instanceId that uniquely identifes this instances of AtLeastOnceApp. + // It will be randomly generated if not provided by the user. final String instanceId = getConfig().getInstanceId(); log.info("instanceId={}", instanceId); + + // Define configuration to connect to Pravega. final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); + + // Create the input and output streams (ignored if they already exist). try (StreamManager streamManager = StreamManager.create(clientConfig)) { streamManager.createScope(getConfig().getScope()); final StreamConfiguration streamConfig = StreamConfiguration.builder() @@ -77,15 +84,18 @@ public void run() throws Exception { getConfig().getMembershipSynchronizerStreamName(), StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); } + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(getConfig().getScope(), getConfig().getStream1Name())) .automaticCheckpointIntervalMillis(getConfig().getCheckpointPeriodMs()) .build(); try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig)) { + // Create the Reader Group (ignored if it already exists) readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); + // Create a Pravega stream writer that we will send our processed output to. EventStreamWriter writer = eventStreamClientFactory.createEventWriter( getConfig().getStream2Name(), new JSONSerializer<>(new TypeToken(){}.getType()), @@ -102,6 +112,17 @@ public void run() throws Exception { Executors.newScheduledThreadPool(1), getConfig().getHeartbeatIntervalMillis(), 1000) { + + /** + * Process an event that was read. + * Processing can be performed asynchronously after this method returns. + * This method must be stateless. + * + * For this demonstration, we output the same event that was read but with + * the processedBy field set. + * + * @param eventRead The event read. + */ @Override public void process(EventRead eventRead) { final SampleEvent event = eventRead.getEvent(); @@ -111,6 +132,10 @@ public void process(EventRead eventRead) { writer.writeEvent(event.routingKey, event); } + /** + * If {@link #process} did not completely process prior events, it must do so before returning. + * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. + */ @Override public void flush() { writer.flush(); @@ -118,6 +143,8 @@ public void flush() { }; processor.startAsync(); + + // Add shutdown hook for graceful shutdown. Runtime.getRuntime().addShutdownHook(new Thread(() -> { log.info("Running shutdown hook."); processor.stopAsync(); @@ -125,6 +152,7 @@ public void flush() { processor.awaitTerminated(); log.info("Processor terminated."); })); + processor.awaitTerminated(); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 18ed6ac4..0f26d4a8 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -22,7 +22,7 @@ import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.UTF8StringSerializer; +import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.UUID; @@ -31,7 +31,7 @@ * A simple example that continuously shows the events in a stream. */ public class EventDebugSink { - private static final org.slf4j.Logger log = LoggerFactory.getLogger(EventDebugSink.class); + private static final Logger log = LoggerFactory.getLogger(EventDebugSink.class); private static final int READER_TIMEOUT_MS = 2000; diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index c36eebeb..cbae7577 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -2,8 +2,7 @@ # Overview -The examples in this directory are intended to illustrate -how at-least-once semantics can be achieved with Pravega. +The examples in this directory are intended to illustrate how at-least-once semantics can be achieved with Pravega. These show how to use Pravega directly, without relying on an external framework such as Apache Flink. These examples include: @@ -16,7 +15,7 @@ These examples include: This application demonstrates reading events from a Pravega stream (stream1), processing each event, and writing each output event to another Pravega stream (stream2). It guarantees that each event is processed at least once, even in the presence of failures. - If multiple instances of this application are executed using the same reader group name, + If multiple instances of this application are executed using the same Reader Group name, each instance will get a distinct subset of events, providing load balancing and redundancy. The user-defined processing function must be stateless. @@ -26,12 +25,11 @@ These examples include: Reader Groups and State Synchronizers. - [EventDebugSink](EventDebugSink.java): - This application reads events from stream2 and displays them - on the console. + This application reads events from stream2 and displays them on the console. # How to Run -- (Optional) Enable INFO level logging by editing the file [logback.xml](../../../../../resources/logback.xml). +- (Optional) Enable INFO (or DEBUG) level logging by editing the file [logback.xml](../../../../../resources/logback.xml). Update it to include: ``` @@ -56,26 +54,118 @@ These examples include: PRAVEGA_SCOPE=examples PRAVEGA_CONTROLLER=tcp://localhost:9090 ./gradlew pravega-client-examples:startEventGenerator ``` - See [Parameters.java](Parameters.java) for available appConfiguration. + See [AppConfiguration.java](AppConfiguration.java) for available parameters. - In another window, start one or more instances of the stream processor. - The `runAtLeastOnceApp.sh` can be used to run multiple instances concurrently. + The `runAtLeastOnceApp.sh` script can be used to run multiple instances concurrently. + ```shell script - cd ~/pravega-samples/pravega-client-examples - ./runAtLeastOnceApp.sh 2 + cd ~/pravega-samples + scripts/runAtLeastOnceApp.sh 2 ``` + + You may view the log files `tmp/atLeastOnceApp-*.log`. -- In another window, start the event debug sink: +- Start the event debug sink: ```shell script ./gradlew pravega-client-examples:startEventDebugSink ``` -# Achieving At-Least-Once Semantics +# Parallelism -Pravega has a sophisticated concept called a -[Reader Group](http://pravega.io/docs/latest/reader-group-design/). +A Reader Group will provide load balancing by assigning each reader a distinct set of segments (partitions). +Readers may not share segments. +If there are more readers than segments, the extra readers will be idle. +The number of segments is controlled by the stream's scaling policy. +A scaling policy can specify a fixed number of segments or a dynamic number of segments based on +the target write rate, measured as events/sec or bytes/sec. -The complete state of a Reader Group is maintained by each reader. -Each reader reads updates from the Reader Group stream. +# Achieving At-Least-Once Semantics -TODO: Finish this section. +## Reader Groups + +Pravega uses a [Reader Group](http://pravega.io/docs/latest/reader-group-design/) to coordinate +load balancing across multiple readers of a Pravega stream. +A Reader Group tracks the position (byte offsets) of each reader. +The Reader Group is the central building block for the at-least-once processor. + +## Checkpoints + +A checkpoint can be initiated on a Reader Group by writing a `ReaderGroupState.CreateCheckpoint` event, +causing all readers to receive a checkpoint event when they call `EventStreamReader.readNextEvent()`. +When a reader receives a checkpoint event, +it is expected to flush any pending writes and then it will notify the Reader Group by writing a +`ReaderGroupState.CheckpointReader` event as represented below. + +```java +class CheckpointReader extends ReaderGroupStateUpdate { + String checkpointId; + String readerId; + Map positions; // map from segment to byte offset of the next event to read + // ... +} +``` + +By default, checkpoints are initiated automatically by the Reader Group. +The interval can be specified using `ReaderGroupConfig.automaticCheckpointIntervalMillis()`. + +## Graceful shutdown + +A graceful shutdown of an at-least-once process requires that it call +`ReaderGroup.readerOffline(readerId, lastPosition)`, where `lastPosition` +is the position object of the last event that was successfully processed. +This method is called automatically when the `EventStreamReader` is closed. +When graceful shutdowns occur, events will be processed exactly once. + +## Ungraceful shutdown + +An ungraceful shutdown of a process can occur when the host abruptly loses power, +when it loses its network connection, or when the process is terminated with `kill -9`. +In these situations, the process is unable to successfully call `ReaderGroup.readerOffline()`. +However, the Reader Group guarantees that any assigned segments will remain assigned +to the reader until `readerOffline()` is called. +If no action is taken, the events in the segments assigned to the dead worker's reader will never be read. +Therefore, we must have another process that is able to detect any dead workers +and call `readerOffline(readerId, lastPosition)` on their behalf. +The `ReaderGroupPruner` class provides this functionality. +Each worker process runs an instance of `ReaderGroupPruner`. +When it detects a dead worker, it calls `readerOffline(readerId, lastPosition)` with +a null value for `lastPosition`. +The null value for `lastPosition` indicates that it should use the position stored in the +last `ReaderGroupState.CheckpointReader` event in the Reader Group. +Any events processed by the now-dead worker after the last checkpoint will be reprocessed by other workers. +If writes are not idempotent, this will produce duplicates. + +In order to detect dead workers, each worker process must run an instance of `ReaderGroupPruner`. +`ReaderGroupPruner` uses a [MembershipSynchronizer](MembershipSynchronizer.java) which uses a +[State Synchronizer](http://pravega.io/docs/latest/state-synchronizer-design/) to +maintain the set of workers that are providing heart beats. +Each worker sends a heart beat by adding an update to the `MembershipSynchronizer`'s State Synchronizer. +Workers that fail to provide a heart beat after 10 intervals will be removed from the `MembershipSynchronizer`. +Finally, any workers in the Reader Group that are not in the `MembershipSynchronizer` are +considered dead and `readerOffline(readerId, null)` will be called by one or more instance of `ReaderGroupPruner`. + +## Stream Life Cycle + +The first instance of `AtLeastOnceApp` will create new State Synchronizer streams for the Reader Group and the `MembershipSynchronizer`. +All subsequent instances will use the existing streams. +It is expected that the number of worker processes may scale up and down and may even scale to zero processes during maintenance windows. +To ensure that the position of readers is not lost, the `AtLastOnceApp` will not automatically delete these streams. +These streams can be deleted manually when it is known that the application will not need to restart from the +stored positions. + +# Achieving Exactly Once Semantics + +Exactly-once semantics can be achieved by using an idempotent writer with an at-least-once processor. +The `AtLeastOnceApp` writes its output to a Pravega stream using `EventStreamWriter` which is *not* idempotent. +However, if this were modified to write to a key/value store or relational database +with a deterministic key, then the writer would be idempotent and the system would provide exactly-once semantics. + +# Stateful Exactly-once Semantics with Apache Flink + +Although achieving at-least-once semantics with a parallel stateless processor is relatively simple +as shown here, it becomes significantly more complex when state must be managed +or when exactly-once semantics is required without an idempotent writer. +In these cases, [Apache Flink](https://flink.apache.org/) with the +[Pravega connectors](https://github.com/pravega/flink-connectors) provides a framework +to greatly simplify application development. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 4c1cedcc..360644d7 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -33,7 +33,9 @@ public class ReaderGroupPruner extends AbstractService implements AutoCloseable private ScheduledFuture task; - public static ReaderGroupPruner create(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis) { + public static ReaderGroupPruner create(ReaderGroup readerGroup, String membershipSynchronizerStreamName, + String readerId, SynchronizerClientFactory clientFactory, + ScheduledExecutorService executor, long heartbeatIntervalMillis) { final ReaderGroupPruner pruner = new ReaderGroupPruner(readerGroup, membershipSynchronizerStreamName, readerId, clientFactory, executor, heartbeatIntervalMillis); pruner.startAsync(); @@ -41,7 +43,8 @@ public static ReaderGroupPruner create(ReaderGroup readerGroup, String membershi return pruner; } - public ReaderGroupPruner(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, SynchronizerClientFactory clientFactory, + public ReaderGroupPruner(ReaderGroup readerGroup, String membershipSynchronizerStreamName, String readerId, + SynchronizerClientFactory clientFactory, ScheduledExecutorService executor, long heartbeatIntervalMillis) { this.readerGroup = readerGroup; this.membershipSynchronizer = new MembershipSynchronizer( @@ -61,8 +64,8 @@ public void run() { try { Set rgMembers = readerGroup.getOnlineReaders(); Set msMembers = membershipSynchronizer.getCurrentMembers(); - log.info("rgMembers={}", rgMembers); - log.info("msMembers={}", msMembers); + log.debug("rgMembers={}", rgMembers); + log.debug("msMembers={}", msMembers); rgMembers.removeAll(msMembers); rgMembers.forEach(readerId -> { log.info("Removing dead reader {} from reader group {}", readerId, readerGroup.getGroupName()); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java index 53368215..2eeeb08e 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; public class SampleEvent { From 13076f2324a7626176657f13603863b1d22b36f0 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 05:50:05 +0000 Subject: [PATCH 30/78] Fix typo. Signed-off-by: Claudio Fahey --- .../src/main/java/io/pravega/example/streamprocessing/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index cbae7577..ed7bc55c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -143,7 +143,7 @@ maintain the set of workers that are providing heart beats. Each worker sends a heart beat by adding an update to the `MembershipSynchronizer`'s State Synchronizer. Workers that fail to provide a heart beat after 10 intervals will be removed from the `MembershipSynchronizer`. Finally, any workers in the Reader Group that are not in the `MembershipSynchronizer` are -considered dead and `readerOffline(readerId, null)` will be called by one or more instance of `ReaderGroupPruner`. +considered dead and `readerOffline(readerId, null)` will be called by one or more instances of `ReaderGroupPruner`. ## Stream Life Cycle From bf5a9b024e5d99854bbfdeeb33033e34dd39304e Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 21 Jun 2020 06:00:14 +0000 Subject: [PATCH 31/78] Fix /README.md. Signed-off-by: Claudio Fahey --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index dec7f028..ab2e6662 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ more complex applications as sub-projects, which show use-cases exploiting one o | `noop` | Example of how to add a simple callback executed upon a read event. | [Java](pravega-client-examples/src/main/java/io/pravega/example/noop) | `statesynchronizer` | Application that allows users to work with `StateSynchronizer` API via CLI. | [Java](pravega-client-examples/src/main/java/io/pravega/example/statesynchronizer) | `streamcuts` | Application examples demonstrating the use of `StreamCut`s via CLI. | [Java](pravega-client-examples/src/main/java/io/pravega/example/streamcuts) -| `streamprocessing` | An example that illustrates exactly-once processing using the Pravega API. | [Java](pravega-client-examples/src/main/java/io/pravega/example/streamprocessing) +| `streamprocessing` | An example that illustrates at-least-once processing using the Pravega API. | [Java](pravega-client-examples/src/main/java/io/pravega/example/streamprocessing) The related documentation and instructions are [here](pravega-client-examples). From 7836d279a444e68221b388ebea2367b1c13f22fa Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 5 Jul 2020 05:39:30 +0000 Subject: [PATCH 32/78] Add integration test (incomplete) Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 2 + .../example/streamprocessing/Testing.md | 22 ++ .../src/main/resources/logback.xml | 2 +- .../streamprocessing/ProcessorGroup.java | 4 + .../StreamProcessingTest.java | 136 +++++++ .../example/streamprocessing/TestEvent.java | 29 ++ .../streamprocessing/TestEventGenerator.java | 26 ++ .../streamprocessing/TestEventValidator.java | 15 + .../utils/EventStreamReaderIterator.java | 25 ++ .../java/io/pravega/utils/SetupUtils.java | 361 ++++++++++++++++++ 10 files changed, 621 insertions(+), 1 deletion(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 892c12d4..06ed3897 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -26,6 +26,8 @@ ext { dependencies { testCompile "junit:junit:${junitVersion}" + testCompile "io.pravega:pravega-standalone:${pravegaVersion}" + compileOnly "org.projectlombok:lombok:${lombokVersion}" compile "io.pravega:pravega-client:${pravegaVersion}", "io.pravega:pravega-common:${pravegaVersion}", diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md new file mode 100644 index 00000000..c9d032f5 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md @@ -0,0 +1,22 @@ + +# Ungraceful shutdown + +- Start multiple JVMs for processors +- Write events +- Read processed events +- Kill -9 JVMs +- Start new JVMs for processors +- Validate that all events are processed at least once. +- Validate that, if an idempotent writer were used (IdempotentEventWriter), all events would be processed exactly once. +- Ordering guarantee: maintains per-routing key ordering, with rewind. + +# Graceful shutdown + +- Same as above but use normal kill. +- Validate exactly once + +# Temporary network partition + +- Must block access from JVM to Pravega. +- Run in Docker and use iptables to block IPs? +- Must create Docker image with this application. \ No newline at end of file diff --git a/pravega-client-examples/src/main/resources/logback.xml b/pravega-client-examples/src/main/resources/logback.xml index ae1a2a69..34f9c7c0 100644 --- a/pravega-client-examples/src/main/resources/logback.xml +++ b/pravega-client-examples/src/main/resources/logback.xml @@ -18,7 +18,7 @@ - + diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java new file mode 100644 index 00000000..b22cc086 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java @@ -0,0 +1,4 @@ +package io.pravega.example.streamprocessing; + +public class ProcessorGroup { +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java new file mode 100644 index 00000000..a8f955da --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -0,0 +1,136 @@ +package io.pravega.example.streamprocessing; + +import com.google.common.collect.Iterators; +import com.google.gson.reflect.TypeToken; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.Serializer; +import io.pravega.utils.EventStreamReaderIterator; +import io.pravega.utils.SetupUtils; +import lombok.Cleanup; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; + +public class StreamProcessingTest { + static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); + + protected static final AtomicReference SETUP_UTILS = new AtomicReference<>(); + + @BeforeClass + public static void setup() throws Exception { + SETUP_UTILS.set(new SetupUtils()); + SETUP_UTILS.get().startAllServices(); + } + + @AfterClass + public static void tearDown() throws Exception { + SETUP_UTILS.get().stopAllServices(); + } + +// public static void runWorker(final ClientConfig clientConfig, +// final String inputStreamName, +// final String readerGroup) throws Exception { +// +// } + + @Test + public void basicTest() throws Exception { + final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); + log.info("Test case: {}", methodName); + + // Prepare writer that will write to the stream that will be the input to the processor. + final String scope = SETUP_UTILS.get().getScope(); + final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); + @Cleanup + final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + final String inputStreamName = "stream-" + UUID.randomUUID().toString(); + final Serializer serializer = new JSONSerializer<>(new TypeToken(){}.getType()); + SETUP_UTILS.get().createTestStream(inputStreamName, 6); + final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); + @Cleanup + final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); + + // Prepare reader that will read from the stream that will be the output from the processor. + final String outputStreamName = inputStreamName; + final String readerGroup = "rg" + UUID.randomUUID().toString().replace("-", ""); + final String readerId = "reader-" + UUID.randomUUID().toString(); + final ReaderConfig readerConfig = ReaderConfig.builder().build(); + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(SETUP_UTILS.get().getStream(outputStreamName)) + .build(); + @Cleanup + final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + @Cleanup + EventStreamReader reader = clientFactory.createReader( + readerId, + readerGroup, + new JSONSerializer<>(new TypeToken(){}.getType()), + ReaderConfig.builder().build()); + + // Create streams with specified segments. + // Create event generator instance. + TestEventGenerator generator = new TestEventGenerator(12); + // Create event validator instance. + TestEventValidator validator = new TestEventValidator(generator); + // Create processor group instance. + ProcessorGroup processorGroup; + // Write 10 historical events. + Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(event.routingKey, event)); + // Start processors. +// processorGroup.start(new int[]{0, 1}); + // Read events from output stream. Return when complete or throw exception if out of order or timeout. + EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); + validator.validate(readerIterator); + // Kill some processors. Start some new ones. +// processorGroup.gracefulStop(new int[]{0, 1}); + // Write events. +// generator.write(10); + // Read events. +// validator.validate(); + +// try (StreamManager streamManager = StreamManager.create(clientConfig)) { +// streamManager.createScope(scope); +// +// StreamConfiguration streamConfig = StreamConfiguration.builder() +// .scalingPolicy(ScalingPolicy.fixed(3)) +// .build(); +// streamManager.createStream(scope, inputStreamName, streamConfig); +// +// try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { +// readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); +// } +// try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); +// final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { +// +// inputWriter.writeEvent("CLAUDIO1"); +// +// AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { +// @Override +// public EventStreamReader createReader() { +// return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); +// } +// +// @Override +// public void write(EventRead eventRead) { +// } +// }; +// processor.call(); +// // TODO: Wait for correct result and then terminate. +// } +// } + log.info("SUCCESS"); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java new file mode 100644 index 00000000..bf626e23 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java @@ -0,0 +1,29 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +public class TestEvent { + public long sequenceNumber; + public String routingKey; + + public TestEvent(long sequenceNumber, String routingKey) { + this.sequenceNumber = sequenceNumber; + this.routingKey = routingKey; + } + + @Override + public String toString() { + return "TestEvent{" + + "sequenceNumber=" + String.format("%6d", sequenceNumber) + + ", routingKey=" + routingKey + + '}'; + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java new file mode 100644 index 00000000..3ab44696 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java @@ -0,0 +1,26 @@ +package io.pravega.example.streamprocessing; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +public class TestEventGenerator implements Iterator { + private final int numKeys; + // map from routing key to sequence number + private final Map lastSequenceNumbers; + + public TestEventGenerator(int numKeys) { + this.numKeys = numKeys; + lastSequenceNumbers = new HashMap(); + } + + @Override + public boolean hasNext() { + return true; + } + + @Override + public TestEvent next() { + return new TestEvent(0, "0"); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java new file mode 100644 index 00000000..ceca8877 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -0,0 +1,15 @@ +package io.pravega.example.streamprocessing; + +import java.util.Iterator; + +public class TestEventValidator { + private final TestEventGenerator generator; + + public TestEventValidator(TestEventGenerator generator) { + this.generator = generator; + } + + public void validate(Iterator events) { + + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java new file mode 100644 index 00000000..a97d7ff9 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -0,0 +1,25 @@ +package io.pravega.utils; + +import io.pravega.client.stream.EventStreamReader; + +import java.util.Iterator; + +public class EventStreamReaderIterator implements Iterator { + private final EventStreamReader reader; + private final long timeoutMillis; + + public EventStreamReaderIterator(EventStreamReader reader, long timeoutMillis) { + this.reader = reader; + this.timeoutMillis = timeoutMillis; + } + + @Override + public boolean hasNext() { + return true; + } + + @Override + public T next() { + return reader.readNextEvent(timeoutMillis).getEvent(); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java new file mode 100644 index 00000000..5ba1fbac --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java @@ -0,0 +1,361 @@ +/* + * Copyright 2019 Flavio Junqueira + * + * Licensed 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 io.pravega.utils; + + import com.google.common.base.Preconditions; + import io.pravega.client.ClientConfig; + import io.pravega.client.EventStreamClientFactory; + import io.pravega.client.admin.StreamManager; + import io.pravega.client.stream.ScalingPolicy; + import io.pravega.client.stream.Stream; + import io.pravega.client.stream.StreamConfiguration; + import io.pravega.client.stream.impl.Controller; + import io.pravega.client.stream.impl.ControllerImpl; + import io.pravega.client.stream.impl.ControllerImplConfig; + import io.pravega.common.concurrent.ExecutorServiceHelpers; + import io.pravega.local.InProcPravegaCluster; + import lombok.Cleanup; + import lombok.Getter; + import lombok.Setter; + import org.apache.commons.lang3.RandomStringUtils; + import org.slf4j.Logger; + import org.slf4j.LoggerFactory; + + import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; + import java.io.InputStream; + import java.net.ServerSocket; + import java.net.URI; + import java.nio.file.Files; + import java.nio.file.Path; + import java.nio.file.StandardCopyOption; + import java.util.concurrent.ScheduledExecutorService; + import java.util.concurrent.atomic.AtomicBoolean; + import java.util.concurrent.atomic.AtomicInteger; + +/** + * Utility functions for creating the test setup. + */ +//@Slf4j +@NotThreadSafe +public final class SetupUtils { + + static Logger log = LoggerFactory.getLogger(SetupUtils.class); + private static final ScheduledExecutorService DEFAULT_SCHEDULED_EXECUTOR_SERVICE = ExecutorServiceHelpers.newScheduledThreadPool(3, "SetupUtils"); + + private final PravegaGateway gateway; + + // Manage the state of the class. + private final AtomicBoolean started = new AtomicBoolean(false); + + // auth enabled by default. Set it to false to disable Pravega authentication and authorization. + @Setter + private boolean enableAuth = false; + + // Set to true to enable TLS + @Setter + private boolean enableTls = false; + + @Setter + private boolean enableHostNameValidation = false; + + private boolean enableRestServer = true; + + // The test Scope name. + @Getter + private final String scope = RandomStringUtils.randomAlphabetic(20); + + public SetupUtils() { + this(System.getProperty("pravega.uri")); + } + + public SetupUtils(String externalUri) { + log.info("SetupUtils constructor"); + if (externalUri != null) { + log.info("Using Pravega services at {}.", externalUri); + gateway = new ExternalPravegaGateway(URI.create(externalUri)); + } else { + log.info("Starting in-process Pravega services."); + gateway = new InProcPravegaGateway(); + } + log.info("Done with constructor {}", gateway.toString()); + } + + + /** + * Start all pravega related services required for the test deployment. + * + * @throws Exception on any errors. + */ + public void startAllServices() throws Exception { + log.info("Starting all services"); + if (!this.started.compareAndSet(false, true)) { + log.warn("Services already started, not attempting to start again"); + return; + } + log.info("Starting gateway"); + gateway.start(); + } + + /** + * Stop the pravega cluster and release all resources. + * + * @throws Exception on any errors. + */ + public void stopAllServices() throws Exception { + if (!this.started.compareAndSet(true, false)) { + log.warn("Services not yet started or already stopped, not attempting to stop"); + return; + } + + try { + gateway.stop(); + } catch (Exception e) { + log.warn("Services did not stop cleanly (" + e.getMessage() + ")", e); + } + } + + /** + * Get resources as temp file. + * + * @param resourceName Name of the resource. + * + * @return Path of the temp file. + */ + static String getFileFromResource(String resourceName) { + try { + Path tempPath = Files.createTempFile("test-", ".tmp"); + tempPath.toFile().deleteOnExit(); + try (InputStream stream = SetupUtils.class.getClassLoader().getResourceAsStream(resourceName)) { + Files.copy(SetupUtils.class.getClassLoader().getResourceAsStream(resourceName), tempPath, StandardCopyOption.REPLACE_EXISTING); + } + return tempPath.toFile().getAbsolutePath(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + /** + * Fetch the controller endpoint for this cluster. + * + * @return URI The controller endpoint to connect to this cluster. + */ + public URI getControllerUri() { + return getClientConfig().getControllerURI(); + } + + /** + * Fetch the client configuration with which to connect to the controller. + */ + public ClientConfig getClientConfig() { + return this.gateway.getClientConfig(); + } + + /** + * Create a controller facade for this cluster. + * @return The controller facade, which must be closed by the caller. + */ + public Controller newController() { + ControllerImplConfig config = ControllerImplConfig.builder() + .clientConfig(getClientConfig()) + .build(); + return new ControllerImpl(config, DEFAULT_SCHEDULED_EXECUTOR_SERVICE); + } + + /** + * Create a {@link EventStreamClientFactory} for this cluster and scope. + */ + public EventStreamClientFactory newClientFactory() { + return EventStreamClientFactory.withScope(this.scope, getClientConfig()); + } + + /** + * Create the test stream. + * + * @param streamName Name of the test stream. + * @param numSegments Number of segments to be created for this stream. + * + * @throws Exception on any errors. + */ + public void createTestStream(final String streamName, final int numSegments) + throws Exception { + Preconditions.checkState(this.started.get(), "Services not yet started"); + Preconditions.checkNotNull(streamName); + Preconditions.checkArgument(numSegments > 0); + + @Cleanup + StreamManager streamManager = StreamManager.create(getClientConfig()); + streamManager.createScope(this.scope); + streamManager.createStream(this.scope, streamName, + StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.fixed(numSegments)) + .build()); + log.info("Created stream: " + streamName); + } + + /** + * Get the stream. + * + * @param streamName Name of the test stream. + * + * @return a Stream + */ + public Stream getStream(final String streamName) { + return Stream.of(this.scope, streamName); + } + + /** + * A gateway interface to Pravega for integration test purposes. + */ + private interface PravegaGateway { + /** + * Starts the gateway. + */ + void start() throws Exception; + + /** + * Stops the gateway. + */ + void stop() throws Exception; + + /** + * Gets the client configuration with which to connect to the controller. + */ + ClientConfig getClientConfig(); + } + + class InProcPravegaGateway implements PravegaGateway { + + // The pravega cluster. + private InProcPravegaCluster inProcPravegaCluster = null; + + @Override + public void start() throws Exception { + log.info("Starting gateway"); + int zkPort = PortUtils.getAvailableListenPort(); + int controllerPort = PortUtils.getAvailableListenPort(); + int hostPort = PortUtils.getAvailableListenPort(); + int restPort = PortUtils.getAvailableListenPort(); + + log.info("Building"); + this.inProcPravegaCluster = InProcPravegaCluster.builder() + .isInProcZK(true) + .secureZK(enableTls) //configure ZK for security + .zkUrl("localhost:" + zkPort) + .zkPort(zkPort) + .isInMemStorage(true) + .isInProcController(true) + .controllerCount(1) + .restServerPort(restPort) + .enableRestServer(enableRestServer) + .isInProcSegmentStore(true) + .segmentStoreCount(1) + .containerCount(4) + .enableMetrics(false) + .enableAuth(enableAuth) + .enableTls(enableTls) + //.certFile(getFileFromResource(CERT_FILE)) // pravega #2519 + //.keyFile(getFileFromResource(KEY_FILE)) + //.jksKeyFile(getFileFromResource(STANDALONE_KEYSTORE_FILE)) + //.jksTrustFile(getFileFromResource(STANDALONE_TRUSTSTORE_FILE)) + //.keyPasswordFile(getFileFromResource(STANDALONE_KEYSTORE_PASSWD_FILE)) + //.passwdFile(getFileFromResource(PASSWD_FILE)) + //.userName(PRAVEGA_USERNAME) + //.passwd(PRAVEGA_PASSWORD) + .build(); + log.info("Done building"); + this.inProcPravegaCluster.setControllerPorts(new int[]{controllerPort}); + this.inProcPravegaCluster.setSegmentStorePorts(new int[]{hostPort}); + this.inProcPravegaCluster.start(); + log.info("Initialized Pravega Cluster"); + log.info("Controller port is {}", controllerPort); + log.info("Host port is {}", hostPort); + log.info("REST server port is {}", restPort); + } + + @Override + public void stop() throws Exception { + inProcPravegaCluster.close(); + } + + @Override + public ClientConfig getClientConfig() { + log.info("Getting client config"); + return ClientConfig.builder() + .controllerURI(URI.create(inProcPravegaCluster.getControllerURI())) + //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) + //.validateHostName(enableHostNameValidation) + //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) + .build(); + } + } + + class ExternalPravegaGateway implements PravegaGateway { + + private final URI controllerUri; + + public ExternalPravegaGateway(URI controllerUri) { + this.controllerUri = controllerUri; + } + + @Override + public void start() throws Exception { + } + + @Override + public void stop() throws Exception { + } + + @Override + public ClientConfig getClientConfig() { + return ClientConfig.builder() + .controllerURI(controllerUri) + //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) + //.validateHostName(enableHostNameValidation) + //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) + .build(); + } + } + + static class PortUtils { + // Linux uses ports from range 32768 - 61000. + private static final int BASE_PORT = 32768; + private static final int MAX_PORT_COUNT = 28232; + private static final AtomicInteger NEXT_PORT = new AtomicInteger(1); + + /** + * A helper method to get a random free port. + * + * @return free port. + */ + public static int getAvailableListenPort() { + for (int i = 0; i < MAX_PORT_COUNT; i++) { + int candidatePort = BASE_PORT + NEXT_PORT.getAndIncrement() % MAX_PORT_COUNT; + try { + ServerSocket serverSocket = new ServerSocket(candidatePort); + serverSocket.close(); + return candidatePort; + } catch (IOException e) { + // Do nothing. Try another port. + } + } + throw new IllegalStateException( + String.format("Could not assign port in range %d - %d", BASE_PORT, MAX_PORT_COUNT + BASE_PORT)); + } + } +} From 5629ffc47c56da0e3ea84cee8c1f0332b16bdfd8 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 5 Jul 2020 21:04:16 +0000 Subject: [PATCH 33/78] Add integration test (incomplete) Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 1 + .../streamprocessing/EventValidatorTest.java | 30 +++++++++++ .../StreamProcessingTest.java | 53 +++---------------- .../example/streamprocessing/TestEvent.java | 10 ++-- .../streamprocessing/TestEventGenerator.java | 21 +++++++- .../streamprocessing/TestEventValidator.java | 43 ++++++++++++++- 6 files changed, 105 insertions(+), 53 deletions(-) create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 06ed3897..2d8e010a 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -27,6 +27,7 @@ ext { dependencies { testCompile "junit:junit:${junitVersion}" testCompile "io.pravega:pravega-standalone:${pravegaVersion}" + testCompile "io.pravega:pravega-test-testcommon:${pravegaVersion}" compileOnly "org.projectlombok:lombok:${lombokVersion}" compile "io.pravega:pravega-client:${pravegaVersion}", diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java new file mode 100644 index 00000000..c55d3996 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java @@ -0,0 +1,30 @@ +package io.pravega.example.streamprocessing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import io.pravega.test.common.AssertExtensions; + +import java.util.Iterator; + +public class EventValidatorTest { + static final Logger log = LoggerFactory.getLogger(EventValidatorTest.class); + + @Test + public void basicTest() throws Exception { + TestEventGenerator generator = new TestEventGenerator(6); + final Iterator generated = ImmutableList.copyOf(Iterators.limit(generator, 100)).iterator(); + TestEventValidator validator = new TestEventValidator(generator); + validator.validate(generated); + } + + @Test + public void missingEventTest() throws Exception { + TestEventGenerator generator = new TestEventGenerator(6); + final Iterator generated = ImmutableList.copyOf(Iterators.limit(generator, 10)).iterator(); + TestEventValidator validator = new TestEventValidator(generator); + AssertExtensions.assertThrows(Exception.class, () -> validator.validate(Iterators.limit(generated, 9))); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index a8f955da..905cb7c2 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -39,12 +39,6 @@ public static void tearDown() throws Exception { SETUP_UTILS.get().stopAllServices(); } -// public static void runWorker(final ClientConfig clientConfig, -// final String inputStreamName, -// final String readerGroup) throws Exception { -// -// } - @Test public void basicTest() throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); @@ -78,59 +72,28 @@ public void basicTest() throws Exception { readerId, readerGroup, new JSONSerializer<>(new TypeToken(){}.getType()), - ReaderConfig.builder().build()); + readerConfig); + EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); // Create streams with specified segments. // Create event generator instance. - TestEventGenerator generator = new TestEventGenerator(12); + TestEventGenerator generator = new TestEventGenerator(6); // Create event validator instance. TestEventValidator validator = new TestEventValidator(generator); // Create processor group instance. ProcessorGroup processorGroup; // Write 10 historical events. - Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(event.routingKey, event)); + Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); // Start processors. // processorGroup.start(new int[]{0, 1}); // Read events from output stream. Return when complete or throw exception if out of order or timeout. - EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); validator.validate(readerIterator); // Kill some processors. Start some new ones. // processorGroup.gracefulStop(new int[]{0, 1}); - // Write events. -// generator.write(10); - // Read events. -// validator.validate(); - -// try (StreamManager streamManager = StreamManager.create(clientConfig)) { -// streamManager.createScope(scope); -// -// StreamConfiguration streamConfig = StreamConfiguration.builder() -// .scalingPolicy(ScalingPolicy.fixed(3)) -// .build(); -// streamManager.createStream(scope, inputStreamName, streamConfig); -// -// try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig)) { -// readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); -// } -// try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); -// final EventStreamWriter inputWriter = clientFactory.createEventWriter( inputStreamName, serializer, eventWriterConfig)) { -// -// inputWriter.writeEvent("CLAUDIO1"); -// -// AtLeastOnceProcessor processor = new AtLeastOnceProcessor() { -// @Override -// public EventStreamReader createReader() { -// return clientFactory.createReader(readerId, readerGroup, serializer, readerConfig); -// } -// -// @Override -// public void write(EventRead eventRead) { -// } -// }; -// processor.call(); -// // TODO: Wait for correct result and then terminate. -// } -// } + Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + validator.validate(readerIterator); + Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + validator.validate(readerIterator); log.info("SUCCESS"); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java index bf626e23..60df7f19 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java @@ -11,19 +11,19 @@ package io.pravega.example.streamprocessing; public class TestEvent { + public int key; public long sequenceNumber; - public String routingKey; - public TestEvent(long sequenceNumber, String routingKey) { + public TestEvent(int key, long sequenceNumber) { + this.key = key; this.sequenceNumber = sequenceNumber; - this.routingKey = routingKey; } @Override public String toString() { return "TestEvent{" + - "sequenceNumber=" + String.format("%6d", sequenceNumber) + - ", routingKey=" + routingKey + + "key=" + key + + ", sequenceNumber=" + sequenceNumber + '}'; } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java index 3ab44696..541eca89 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java @@ -1,17 +1,25 @@ package io.pravega.example.streamprocessing; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.HashMap; import java.util.Iterator; import java.util.Map; public class TestEventGenerator implements Iterator { + static final Logger log = LoggerFactory.getLogger(TestEventGenerator.class); + private final int numKeys; + private int lastKey; // map from routing key to sequence number private final Map lastSequenceNumbers; public TestEventGenerator(int numKeys) { this.numKeys = numKeys; - lastSequenceNumbers = new HashMap(); + this.lastKey = numKeys - 1; + lastSequenceNumbers = new HashMap<>(); +// IntStream.range(0, numKeys).forEach(key -> lastSequenceNumbers.put(key, 0L)); } @Override @@ -21,6 +29,15 @@ public boolean hasNext() { @Override public TestEvent next() { - return new TestEvent(0, "0"); + lastKey = (lastKey + 1) % numKeys;; + final Long sequenceNumber = lastSequenceNumbers.getOrDefault(lastKey, -1L) + 1; + lastSequenceNumbers.put(lastKey, sequenceNumber); + final TestEvent event = new TestEvent(lastKey, sequenceNumber); + log.info("event={}", event); + return event; + } + + public Map getLastSequenceNumbers() { + return new HashMap<>(lastSequenceNumbers); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index ceca8877..d5219d44 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -1,15 +1,56 @@ package io.pravega.example.streamprocessing; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; import java.util.Iterator; +import java.util.Map; public class TestEventValidator { + static final Logger log = LoggerFactory.getLogger(TestEventValidator.class); + private final TestEventGenerator generator; + // map from routing key to sequence number + private final Map receivedSequenceNumbers; public TestEventValidator(TestEventGenerator generator) { this.generator = generator; + receivedSequenceNumbers = new HashMap<>(); } public void validate(Iterator events) { - + // pendingSequenceNumbers contains a map from key to sequence number for events that have been generated but not yet received by validate. + // A key is removed when all events up to the generated sequence number for that key are received. + final Map pendingSequenceNumbers = new HashMap<>(); + final Map generatedSequenceNumbers = generator.getLastSequenceNumbers(); + log.info("generatedSequenceNumbers={}", generatedSequenceNumbers); + generatedSequenceNumbers.forEach((key, sequenceNumber) -> { + if (receivedSequenceNumbers.getOrDefault(key, -1L) < sequenceNumber) { + pendingSequenceNumbers.put(key, sequenceNumber); + } + }); + log.info("pendingSequenceNumbers={}", pendingSequenceNumbers); + while (events.hasNext()) { + final TestEvent event = events.next(); + final long lastReceivedSequenceNumber = receivedSequenceNumbers.getOrDefault(event.key, -1L); + log.info("event={}, lastReceivedSequenceNumber={}", event, lastReceivedSequenceNumber); + if (event.sequenceNumber <= lastReceivedSequenceNumber) { + throw new IllegalStateException("Duplicate event"); + } else if (event.sequenceNumber > lastReceivedSequenceNumber + 1) { + throw new IllegalStateException("Gap"); + } else { + receivedSequenceNumbers.put(event.key, event.sequenceNumber); + if (pendingSequenceNumbers.getOrDefault(event.key, -1L) <= event.sequenceNumber) { + pendingSequenceNumbers.remove(event.key); + if (pendingSequenceNumbers.size() == 0) { + // All data received. + log.info("All data received; receivedSequenceNumbers={}", receivedSequenceNumbers); + return; + } + } + } + } + throw new IllegalStateException("No more events"); } } From ca59a618d89238a200aa80c9164c53d1bb16ecf3 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sun, 5 Jul 2020 23:51:28 +0000 Subject: [PATCH 34/78] Add integration test (incomplete) Signed-off-by: Claudio Fahey --- .../streamprocessing/ProcessorGroup.java | 4 - .../StreamProcessingTest.java | 96 ++++++++++++++--- .../example/streamprocessing/TestEvent.java | 3 + .../streamprocessing/WorkerProcess.java | 100 ++++++++++++++++++ .../streamprocessing/WorkerProcessConfig.java | 18 ++++ .../streamprocessing/WorkerProcessGroup.java | 41 +++++++ 6 files changed, 245 insertions(+), 17 deletions(-) delete mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java deleted file mode 100644 index b22cc086..00000000 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/ProcessorGroup.java +++ /dev/null @@ -1,4 +0,0 @@ -package io.pravega.example.streamprocessing; - -public class ProcessorGroup { -} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 905cb7c2..6566e17b 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -11,6 +11,7 @@ import io.pravega.client.stream.ReaderConfig; import io.pravega.client.stream.ReaderGroupConfig; import io.pravega.client.stream.Serializer; +import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; import lombok.Cleanup; @@ -30,7 +31,7 @@ public class StreamProcessingTest { @BeforeClass public static void setup() throws Exception { - SETUP_UTILS.set(new SetupUtils()); + SETUP_UTILS.set(new SetupUtils("tcp://localhost:9090")); SETUP_UTILS.get().startAllServices(); } @@ -40,23 +41,25 @@ public static void tearDown() throws Exception { } @Test - public void basicTest() throws Exception { + public void noProcessorTest() throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); - // Prepare writer that will write to the stream that will be the input to the processor. + // Create stream. final String scope = SETUP_UTILS.get().getScope(); final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); + final String inputStreamName = "stream-" + UUID.randomUUID().toString(); + SETUP_UTILS.get().createTestStream(inputStreamName, 6); @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); - final String inputStreamName = "stream-" + UUID.randomUUID().toString(); + + // Prepare writer that will write to the stream. final Serializer serializer = new JSONSerializer<>(new TypeToken(){}.getType()); - SETUP_UTILS.get().createTestStream(inputStreamName, 6); final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); - // Prepare reader that will read from the stream that will be the output from the processor. + // Prepare reader that will read from the stream. final String outputStreamName = inputStreamName; final String readerGroup = "rg" + UUID.randomUUID().toString().replace("-", ""); final String readerId = "reader-" + UUID.randomUUID().toString(); @@ -68,24 +71,87 @@ public void basicTest() throws Exception { final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); @Cleanup - EventStreamReader reader = clientFactory.createReader( + final EventStreamReader reader = clientFactory.createReader( readerId, readerGroup, new JSONSerializer<>(new TypeToken(){}.getType()), readerConfig); EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); + // Create event generator instance. + final TestEventGenerator generator = new TestEventGenerator(6); + // Create event validator instance. + final TestEventValidator validator = new TestEventValidator(generator); + // Write historical events. + Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + // Read events from output stream. Return when complete or throw exception if out of order or timeout. + validator.validate(readerIterator); + Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + validator.validate(readerIterator); + Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + validator.validate(readerIterator); + log.info("SUCCESS"); + } + + @Test + public void basicTest() throws Exception { + final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); + log.info("Test case: {}", methodName); + + final String scope = SETUP_UTILS.get().getScope(); + final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); + final String inputStreamName = "input-stream-" + UUID.randomUUID().toString(); + final String outputStreamName = "output-stream-" + UUID.randomUUID().toString(); + final String membershipSynchronizerStreamName = "ms-" + UUID.randomUUID().toString(); + final String inputStreamReaderGroupName = "rg" + UUID.randomUUID().toString().replace("-", ""); + + // Start processors. This will also create the necessary streams. + final WorkerProcessConfig workerProcessConfig = WorkerProcessConfig.builder() + .scope(scope) + .clientConfig(clientConfig) + .readerGroupName(inputStreamReaderGroupName) + .inputStreamName(inputStreamName) + .outputStreamName(outputStreamName) + .membershipSynchronizerStreamName(membershipSynchronizerStreamName) + .numSegments(6) + .build(); + final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); + workerProcessGroup.start(new int[]{0}); + + // Prepare generator writer that will write to the stream read by the processor. + @Cleanup + final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + final Serializer serializer = new JSONSerializer<>(new TypeToken(){}.getType()); + final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); + @Cleanup + final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); + + // Prepare validation reader that will read from the stream written by the processor. + final String validationReaderGroupName = "rg" + UUID.randomUUID().toString().replace("-", ""); + final String validationReaderId = "reader-" + UUID.randomUUID().toString(); + final ReaderConfig validationReaderConfig = ReaderConfig.builder().build(); + final ReaderGroupConfig validationReaderGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(scope, outputStreamName)) + .build(); + @Cleanup + final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); + readerGroupManager.createReaderGroup(validationReaderGroupName, validationReaderGroupConfig); + @Cleanup + final EventStreamReader validationReader = clientFactory.createReader( + validationReaderId, + validationReaderGroupName, + new JSONSerializer<>(new TypeToken(){}.getType()), + validationReaderConfig); + EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, 30000); + // Create streams with specified segments. // Create event generator instance. - TestEventGenerator generator = new TestEventGenerator(6); + final TestEventGenerator generator = new TestEventGenerator(6); // Create event validator instance. - TestEventValidator validator = new TestEventValidator(generator); - // Create processor group instance. - ProcessorGroup processorGroup; + final TestEventValidator validator = new TestEventValidator(generator); // Write 10 historical events. Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - // Start processors. -// processorGroup.start(new int[]{0, 1}); + // Read events from output stream. Return when complete or throw exception if out of order or timeout. validator.validate(readerIterator); // Kill some processors. Start some new ones. @@ -94,6 +160,10 @@ public void basicTest() throws Exception { validator.validate(readerIterator); Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); validator.validate(readerIterator); + + validationReader.close(); + readerGroupManager.deleteReaderGroup(validationReaderGroupName); + // TODO: Delete streams. log.info("SUCCESS"); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java index 60df7f19..3bcd35d1 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java @@ -13,10 +13,12 @@ public class TestEvent { public int key; public long sequenceNumber; + public int processedByInstanceId; public TestEvent(int key, long sequenceNumber) { this.key = key; this.sequenceNumber = sequenceNumber; + processedByInstanceId = -1; } @Override @@ -24,6 +26,7 @@ public String toString() { return "TestEvent{" + "key=" + key + ", sequenceNumber=" + sequenceNumber + + ", processedByInstanceId=" + processedByInstanceId + '}'; } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java new file mode 100644 index 00000000..612e27be --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -0,0 +1,100 @@ +package io.pravega.example.streamprocessing; + +import com.google.common.util.concurrent.AbstractExecutionThreadService; +import com.google.gson.reflect.TypeToken; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.SynchronizerClientFactory; +import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ReaderConfig; +import io.pravega.client.stream.ReaderGroup; +import io.pravega.client.stream.ReaderGroupConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Serializer; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import lombok.Builder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Executors; + +@Builder +public class WorkerProcess extends AbstractExecutionThreadService { + private static final Logger log = LoggerFactory.getLogger(WorkerProcess.class); + + private final WorkerProcessConfig config; + private final int instanceId; + + // Create the input, output, and state sychronizer streams (ignored if they already exist). + public void init() { + log.info("init: BEGIN"); + try (StreamManager streamManager = StreamManager.create(config.clientConfig)) { + streamManager.createScope(config.scope); + final StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.fixed(config.numSegments)).build(); + streamManager.createStream(config.scope, config.inputStreamName, streamConfig); + streamManager.createStream(config.scope, config.outputStreamName, streamConfig); + streamManager.createStream( + config.scope, + config.membershipSynchronizerStreamName, + StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); + } + final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() + .stream(Stream.of(config.scope, config.inputStreamName)) + .automaticCheckpointIntervalMillis(config.checkpointPeriodMs) + .build(); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(config.scope, config.clientConfig)) { + // Create the Reader Group (ignored if it already exists) + readerGroupManager.createReaderGroup(config.readerGroupName, readerGroupConfig); + } + log.info("init: END"); + } + + @Override + protected void run() throws Exception { + Serializer serializer = new JSONSerializer<>(new TypeToken() {}.getType()); + try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(config.scope, config.clientConfig)) { + final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(config.readerGroupName); + // Create client factories. + try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(config.scope, config.clientConfig); + SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(config.scope, config.clientConfig); + // Create a Pravega stream writer that we will send our processed output to. + EventStreamWriter writer = eventStreamClientFactory.createEventWriter( + config.outputStreamName, + serializer, + EventWriterConfig.builder().build())) { + + final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + Integer.toString(instanceId), + readerGroup, + config.membershipSynchronizerStreamName, + serializer, + ReaderConfig.builder().build(), + eventStreamClientFactory, + synchronizerClientFactory, + Executors.newScheduledThreadPool(1), + config.heartbeatIntervalMillis, + config.readTimeoutMillis) { + @Override + public void process(EventRead eventRead) { + final TestEvent event = eventRead.getEvent(); + event.processedByInstanceId = instanceId; + log.info("{}", event); + writer.writeEvent(Integer.toString(event.key), event); + } + + @Override + public void flush() { + writer.flush(); + } + }; + processor.startAsync(); + processor.awaitTerminated(); + } + } + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java new file mode 100644 index 00000000..ebbeb04a --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java @@ -0,0 +1,18 @@ +package io.pravega.example.streamprocessing; + +import io.pravega.client.ClientConfig; +import lombok.Builder; + +@Builder +public class WorkerProcessConfig { + public final String scope; + public final ClientConfig clientConfig; + public final String readerGroupName; + public final String inputStreamName; + public final String outputStreamName; + public final String membershipSynchronizerStreamName; + public final int numSegments; + @Builder.Default public final long checkpointPeriodMs = 1000; + @Builder.Default public final long heartbeatIntervalMillis = 1000; + @Builder.Default public final long readTimeoutMillis = 1000; +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java new file mode 100644 index 00000000..299e9005 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -0,0 +1,41 @@ +package io.pravega.example.streamprocessing; + +import lombok.Builder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; + +@Builder +public class WorkerProcessGroup { + private static final Logger log = LoggerFactory.getLogger(WorkerProcessGroup.class); + + private final WorkerProcessConfig config; + private final Map workers = new HashMap<>(); + + /** + * Streams are guaranteed to exist after calling this method. + */ + public void start(int[] instanceIds) { + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + log.info("start: instanceId={}", instanceId); + workers.putIfAbsent(instanceId, WorkerProcess.builder().config(config).instanceId(instanceId).build()); + }); + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + final WorkerProcess worker = workers.get(instanceId); + worker.init(); + worker.startAsync(); + }); + } + + public void stop(int[] instanceIds) { + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + workers.get(instanceId).stopAsync(); + }); + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + workers.get(instanceId).awaitTerminated(); + }); + } +} From 36b18cd3a1ae3bd3017900140b00da6c2dd33783 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Mon, 6 Jul 2020 03:04:35 +0000 Subject: [PATCH 35/78] Basic graceful shutdown test working Signed-off-by: Claudio Fahey --- .../StreamProcessingTest.java | 42 ++++++++++++++----- .../streamprocessing/TestEventValidator.java | 11 ++++- .../streamprocessing/WorkerProcess.java | 20 +++++++-- .../streamprocessing/WorkerProcessGroup.java | 23 +++++++--- .../utils/EventStreamReaderIterator.java | 29 ++++++++++++- 5 files changed, 102 insertions(+), 23 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 6566e17b..9ff1c0d2 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -5,6 +5,7 @@ import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.ReaderGroupManager; +import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.EventStreamReader; import io.pravega.client.stream.EventStreamWriter; import io.pravega.client.stream.EventWriterConfig; @@ -105,7 +106,9 @@ public void basicTest() throws Exception { final String membershipSynchronizerStreamName = "ms-" + UUID.randomUUID().toString(); final String inputStreamReaderGroupName = "rg" + UUID.randomUUID().toString().replace("-", ""); - // Start processors. This will also create the necessary streams. + @Cleanup + StreamManager streamManager = StreamManager.create(clientConfig); + final WorkerProcessConfig workerProcessConfig = WorkerProcessConfig.builder() .scope(scope) .clientConfig(clientConfig) @@ -115,8 +118,11 @@ public void basicTest() throws Exception { .membershipSynchronizerStreamName(membershipSynchronizerStreamName) .numSegments(6) .build(); + @Cleanup final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); - workerProcessGroup.start(new int[]{0}); + + // Start initial set of processors. This will also create the necessary streams. + workerProcessGroup.start(0, 1, 2); // Prepare generator writer that will write to the stream read by the processor. @Cleanup @@ -144,26 +150,42 @@ public void basicTest() throws Exception { validationReaderConfig); EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, 30000); - // Create streams with specified segments. - // Create event generator instance. final TestEventGenerator generator = new TestEventGenerator(6); - // Create event validator instance. final TestEventValidator validator = new TestEventValidator(generator); - // Write 10 historical events. - Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + // Write events to input stream. + Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); // Read events from output stream. Return when complete or throw exception if out of order or timeout. validator.validate(readerIterator); - // Kill some processors. Start some new ones. -// processorGroup.gracefulStop(new int[]{0, 1}); + + // Write and read additional events. Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); validator.validate(readerIterator); Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); validator.validate(readerIterator); + log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); + + workerProcessGroup.start(3); + workerProcessGroup.stop(0, 1, 2); + + Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); + validator.validate(readerIterator); + + log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); + + // Cleanup + log.info("Cleanup"); + workerProcessGroup.close(); validationReader.close(); + readerGroupManager.deleteReaderGroup(inputStreamReaderGroupName); readerGroupManager.deleteReaderGroup(validationReaderGroupName); - // TODO: Delete streams. + streamManager.sealStream(scope, inputStreamName); + streamManager.sealStream(scope, outputStreamName); + streamManager.sealStream(scope, membershipSynchronizerStreamName); + streamManager.deleteStream(scope, inputStreamName); + streamManager.deleteStream(scope, outputStreamName); + streamManager.deleteStream(scope, membershipSynchronizerStreamName); log.info("SUCCESS"); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index d5219d44..59f63378 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -12,11 +12,13 @@ public class TestEventValidator { private final TestEventGenerator generator; // map from routing key to sequence number - private final Map receivedSequenceNumbers; + private final Map receivedSequenceNumbers = new HashMap<>(); + + // map from instanceId to count of events + private final Map eventCountByInstanceId = new HashMap<>(); public TestEventValidator(TestEventGenerator generator) { this.generator = generator; - receivedSequenceNumbers = new HashMap<>(); } public void validate(Iterator events) { @@ -41,6 +43,7 @@ public void validate(Iterator events) { throw new IllegalStateException("Gap"); } else { receivedSequenceNumbers.put(event.key, event.sequenceNumber); + eventCountByInstanceId.merge(event.processedByInstanceId, 1L, Long::sum); // increment counter if (pendingSequenceNumbers.getOrDefault(event.key, -1L) <= event.sequenceNumber) { pendingSequenceNumbers.remove(event.key); if (pendingSequenceNumbers.size() == 0) { @@ -53,4 +56,8 @@ public void validate(Iterator events) { } throw new IllegalStateException("No more events"); } + + public Map getEventCountByInstanceId() { + return eventCountByInstanceId; + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index 612e27be..c3783650 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -21,6 +21,7 @@ import org.slf4j.LoggerFactory; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; @Builder public class WorkerProcess extends AbstractExecutionThreadService { @@ -29,6 +30,8 @@ public class WorkerProcess extends AbstractExecutionThreadService { private final WorkerProcessConfig config; private final int instanceId; + private final AtomicReference> processor = new AtomicReference<>(); + // Create the input, output, and state sychronizer streams (ignored if they already exist). public void init() { log.info("init: BEGIN"); @@ -68,7 +71,7 @@ protected void run() throws Exception { serializer, EventWriterConfig.builder().build())) { - final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + final AtLeastOnceProcessor proc = new AtLeastOnceProcessor( Integer.toString(instanceId), readerGroup, config.membershipSynchronizerStreamName, @@ -92,9 +95,20 @@ public void flush() { writer.flush(); } }; - processor.startAsync(); - processor.awaitTerminated(); + processor.set(proc); + proc.startAsync(); + proc.awaitTerminated(); } } } + + @Override + protected void triggerShutdown() { + log.info("triggerShutdown: BEGIN"); + final AtLeastOnceProcessor proc = processor.getAndSet(null); + if (proc != null) { + proc.stopAsync(); + } + log.info("triggerShutdown: END"); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 299e9005..84b95c30 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -9,7 +9,7 @@ import java.util.stream.IntStream; @Builder -public class WorkerProcessGroup { +public class WorkerProcessGroup implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(WorkerProcessGroup.class); private final WorkerProcessConfig config; @@ -18,8 +18,8 @@ public class WorkerProcessGroup { /** * Streams are guaranteed to exist after calling this method. */ - public void start(int[] instanceIds) { - IntStream.of(instanceIds).parallel().forEach(instanceId -> { + public void start(int... instanceIds) { + IntStream.of(instanceIds).forEach(instanceId -> { log.info("start: instanceId={}", instanceId); workers.putIfAbsent(instanceId, WorkerProcess.builder().config(config).instanceId(instanceId).build()); }); @@ -30,12 +30,23 @@ public void start(int[] instanceIds) { }); } - public void stop(int[] instanceIds) { - IntStream.of(instanceIds).parallel().forEach(instanceId -> { + public void stop(int... instanceIds) { + IntStream.of(instanceIds).forEach(instanceId -> { workers.get(instanceId).stopAsync(); }); - IntStream.of(instanceIds).parallel().forEach(instanceId -> { + IntStream.of(instanceIds).forEach(instanceId -> { workers.get(instanceId).awaitTerminated(); + workers.remove(instanceId); }); } + + public void stopAll() { + log.info("stopAll: workers={}", workers); + stop(workers.keySet().stream().mapToInt(i -> i).toArray()); + } + + @Override + public void close() throws Exception { + stopAll(); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java index a97d7ff9..ee130153 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -1,12 +1,15 @@ package io.pravega.utils; +import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; import java.util.Iterator; +import java.util.Optional; public class EventStreamReaderIterator implements Iterator { private final EventStreamReader reader; private final long timeoutMillis; + private Optional nextEvent = Optional.empty(); public EventStreamReaderIterator(EventStreamReader reader, long timeoutMillis) { this.reader = reader; @@ -15,11 +18,33 @@ public EventStreamReaderIterator(EventStreamReader reader, long timeoutMillis @Override public boolean hasNext() { - return true; + readIfNeeded(); + return nextEvent.isPresent(); } @Override public T next() { - return reader.readNextEvent(timeoutMillis).getEvent(); + readIfNeeded(); + if (nextEvent.isPresent()) { + final T event = nextEvent.get(); + nextEvent = Optional.empty(); + return event; + } else { + throw new RuntimeException("Timeout"); + } + } + + private void readIfNeeded() { + if (!nextEvent.isPresent()) { + for (; ; ) { + final EventRead eventRead = reader.readNextEvent(timeoutMillis); + if (!eventRead.isCheckpoint()) { + if (eventRead.getEvent() != null) { + nextEvent = Optional.of(eventRead.getEvent()); + } + return; + } + } + } } } From b47ed444a74cfe58f0399628259a6fe96613a0c4 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 8 Jul 2020 04:14:45 +0000 Subject: [PATCH 36/78] AtLeastOnceProcessor now uses suppliers. Added basic structure for fault injection. Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 25 ++++--- .../AtLeastOnceProcessor.java | 68 ++++++------------- .../AtLeastOnceProcessorInstrumented.java | 52 ++++++++++++++ .../streamprocessing/WorkerProcess.java | 51 +++++++------- 4 files changed, 111 insertions(+), 85 deletions(-) create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 15867c3c..16aee75a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -102,17 +102,20 @@ public void run() throws Exception { EventWriterConfig.builder().build())) { final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( - instanceId, - readerGroup, - getConfig().getMembershipSynchronizerStreamName(), - new JSONSerializer<>(new TypeToken(){}.getType()), - ReaderConfig.builder().build(), - eventStreamClientFactory, - synchronizerClientFactory, - Executors.newScheduledThreadPool(1), - getConfig().getHeartbeatIntervalMillis(), - 1000) { - + () -> ReaderGroupPruner.create( + readerGroup, + getConfig().getMembershipSynchronizerStreamName(), + instanceId, + synchronizerClientFactory, + Executors.newScheduledThreadPool(1), + getConfig().getHeartbeatIntervalMillis()), + () -> eventStreamClientFactory.createReader( + instanceId, + readerGroup.getGroupName(), + new JSONSerializer<>(new TypeToken(){}.getType()), + ReaderConfig.builder().build()), + 1000) + { /** * Process an event that was read. * Processing can be performed asynchronously after this method returns. diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 706bc8c9..7ce19e21 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -11,19 +11,14 @@ package io.pravega.example.streamprocessing; import com.google.common.util.concurrent.AbstractExecutionThreadService; -import io.pravega.client.EventStreamClientFactory; -import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; import io.pravega.client.stream.EventStreamWriter; import io.pravega.client.stream.Position; -import io.pravega.client.stream.ReaderConfig; -import io.pravega.client.stream.ReaderGroup; -import io.pravega.client.stream.Serializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Supplier; /** * This is an abstract class for implementing a stateless event processor with Pravega. @@ -36,37 +31,13 @@ abstract public class AtLeastOnceProcessor extends AbstractExecutionThreadService { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessor.class); - private final String instanceId; - private final ReaderGroup readerGroup; - private final String membershipSynchronizerStreamName; - private final Serializer serializer; - private final ReaderConfig readerConfig; - private final EventStreamClientFactory eventStreamClientFactory; - private final SynchronizerClientFactory synchronizerClientFactory; - private final ScheduledExecutorService executor; - private final long heartbeatIntervalMillis; + private final Supplier prunerSupplier; + private final Supplier> readerSupplier; private final long readTimeoutMillis; - public AtLeastOnceProcessor( - String instanceId, - ReaderGroup readerGroup, - String membershipSynchronizerStreamName, - Serializer serializer, - ReaderConfig readerConfig, - EventStreamClientFactory eventStreamClientFactory, - SynchronizerClientFactory synchronizerClientFactory, - ScheduledExecutorService executor, - long heartbeatIntervalMillis, - long readTimeoutMillis) { - this.instanceId = instanceId; - this.readerGroup = readerGroup; - this.membershipSynchronizerStreamName = membershipSynchronizerStreamName; - this.serializer = serializer; - this.readerConfig = readerConfig; - this.eventStreamClientFactory = eventStreamClientFactory; - this.synchronizerClientFactory = synchronizerClientFactory; - this.executor = executor; - this.heartbeatIntervalMillis = heartbeatIntervalMillis; + public AtLeastOnceProcessor(Supplier prunerSupplier, Supplier> readerSupplier, long readTimeoutMillis) { + this.prunerSupplier = prunerSupplier; + this.readerSupplier = readerSupplier; this.readTimeoutMillis = readTimeoutMillis; } @@ -79,22 +50,17 @@ public AtLeastOnceProcessor( */ @Override protected void run() throws Exception { - try (final ReaderGroupPruner pruner = ReaderGroupPruner.create( - readerGroup, - membershipSynchronizerStreamName, - instanceId, - synchronizerClientFactory, - executor, - heartbeatIntervalMillis); - final EventStreamReader reader = eventStreamClientFactory.createReader( - instanceId, - readerGroup.getGroupName(), - serializer, - readerConfig)) { - + // It is critical that the ReaderGroupPruner is running (and therefore has added itself to the membership synchronizer) + // before the EventStreamReader is created. Otherwise, another ReaderGroupPruner instance may place this reader offline. + // It is also critical that when this method stops running the ReaderGroupPruner is eventually stopped so that + // it no longer sends heartbeats. + try (final ReaderGroupPruner pruner = prunerSupplier.get(); + final EventStreamReader reader = readerSupplier.get()) { while (isRunning()) { + injectFaultBeforeRead(pruner); final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); log.info("eventRead={}", eventRead); + injectFaultAfterRead(pruner); if (eventRead.isCheckpoint()) { flush(); } else if (eventRead.getEvent() != null) { @@ -126,4 +92,10 @@ protected void run() throws Exception { */ public void flush() { } + + protected void injectFaultBeforeRead(ReaderGroupPruner pruner) throws Exception { + } + + protected void injectFaultAfterRead(ReaderGroupPruner pruner) throws Exception { + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java new file mode 100644 index 00000000..923501d5 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -0,0 +1,52 @@ +package io.pravega.example.streamprocessing; + +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.common.util.ReusableLatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Supplier; + +public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor { + private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessorInstrumented.class); + + private final int instanceId; + private final EventStreamWriter writer; + private final ReusableLatch latch; + + public AtLeastOnceProcessorInstrumented( + Supplier pruner, + Supplier> reader, + long readTimeoutMillis, + int instanceId, + EventStreamWriter writer, + ReusableLatch latch) { + super(pruner, reader, readTimeoutMillis); + this.instanceId = instanceId; + this.writer = writer; + this.latch = latch; + } + + @Override + public void process(EventRead eventRead) { + final TestEvent event = eventRead.getEvent(); + event.processedByInstanceId = instanceId; + log.info("{}", event); + writer.writeEvent(Integer.toString(event.key), event); + } + + @Override + public void flush() { + writer.flush(); + } + + @Override + protected void injectFaultBeforeRead(ReaderGroupPruner pruner) throws Exception { + // Stop pruner (but do not close it). This will also stop the membership synchronizer. + pruner.stopAsync(); + latch.await(); + throw new Exception(); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index c3783650..9818ab99 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -6,7 +6,6 @@ import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.admin.ReaderGroupManager; import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamWriter; import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ReaderConfig; @@ -16,6 +15,7 @@ import io.pravega.client.stream.Serializer; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; +import io.pravega.common.util.ReusableLatch; import lombok.Builder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,10 +29,11 @@ public class WorkerProcess extends AbstractExecutionThreadService { private final WorkerProcessConfig config; private final int instanceId; + private final ReusableLatch latch = new ReusableLatch(true); private final AtomicReference> processor = new AtomicReference<>(); - // Create the input, output, and state sychronizer streams (ignored if they already exist). + // Create the input, output, and state synchronizer streams (ignored if they already exist). public void init() { log.info("init: BEGIN"); try (StreamManager streamManager = StreamManager.create(config.clientConfig)) { @@ -71,30 +72,23 @@ protected void run() throws Exception { serializer, EventWriterConfig.builder().build())) { - final AtLeastOnceProcessor proc = new AtLeastOnceProcessor( - Integer.toString(instanceId), - readerGroup, - config.membershipSynchronizerStreamName, - serializer, - ReaderConfig.builder().build(), - eventStreamClientFactory, - synchronizerClientFactory, - Executors.newScheduledThreadPool(1), - config.heartbeatIntervalMillis, - config.readTimeoutMillis) { - @Override - public void process(EventRead eventRead) { - final TestEvent event = eventRead.getEvent(); - event.processedByInstanceId = instanceId; - log.info("{}", event); - writer.writeEvent(Integer.toString(event.key), event); - } - - @Override - public void flush() { - writer.flush(); - } - }; + final AtLeastOnceProcessor proc = new AtLeastOnceProcessorInstrumented( + () -> ReaderGroupPruner.create( + readerGroup, + config.membershipSynchronizerStreamName, + Integer.toString(instanceId), + synchronizerClientFactory, + Executors.newScheduledThreadPool(1), + config.heartbeatIntervalMillis), + () -> eventStreamClientFactory.createReader( + Integer.toString(instanceId), + readerGroup.getGroupName(), + serializer, + ReaderConfig.builder().build()), + config.readTimeoutMillis, + instanceId, + writer, + latch); processor.set(proc); proc.startAsync(); proc.awaitTerminated(); @@ -109,6 +103,11 @@ protected void triggerShutdown() { if (proc != null) { proc.stopAsync(); } + latch.release(); log.info("triggerShutdown: END"); } + + public void pause() { + latch.reset(); + } } From ad26df4dfb849e1798979d0f46bebb181d80d0c0 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 8 Jul 2020 05:25:11 +0000 Subject: [PATCH 37/78] Add pause and unpause to ReaderGroupPruner Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceProcessor.java | 10 ++++------ .../streamprocessing/ReaderGroupPruner.java | 10 ++++++++++ .../AtLeastOnceProcessorInstrumented.java | 17 +++++++++++------ .../streamprocessing/StreamProcessingTest.java | 7 ++++--- .../streamprocessing/WorkerProcessGroup.java | 12 ++++++++++++ 5 files changed, 41 insertions(+), 15 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 7ce19e21..5af3447c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -57,10 +57,11 @@ protected void run() throws Exception { try (final ReaderGroupPruner pruner = prunerSupplier.get(); final EventStreamReader reader = readerSupplier.get()) { while (isRunning()) { - injectFaultBeforeRead(pruner); final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); log.info("eventRead={}", eventRead); - injectFaultAfterRead(pruner); + // We must inject the fault between read and process. + // This ensures that a *new* event cannot be processed after the fault injection latch is set. + injectFault(pruner); if (eventRead.isCheckpoint()) { flush(); } else if (eventRead.getEvent() != null) { @@ -93,9 +94,6 @@ protected void run() throws Exception { public void flush() { } - protected void injectFaultBeforeRead(ReaderGroupPruner pruner) throws Exception { - } - - protected void injectFaultAfterRead(ReaderGroupPruner pruner) throws Exception { + protected void injectFault(ReaderGroupPruner pruner) throws Exception { } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 360644d7..2fe9745c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -93,6 +93,16 @@ protected void doStop() { membershipSynchronizer.stopAsync(); } + public void pause() { + log.warn("paused"); + task.cancel(false); + } + + public void unpause() { + log.warn("unpause"); +// throw new UnsupportedOperationException(); + } + @Override public void close() throws Exception { stopAsync(); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 923501d5..e7097ba6 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -33,7 +33,7 @@ public AtLeastOnceProcessorInstrumented( public void process(EventRead eventRead) { final TestEvent event = eventRead.getEvent(); event.processedByInstanceId = instanceId; - log.info("{}", event); + log.info("process: event={}", event); writer.writeEvent(Integer.toString(event.key), event); } @@ -43,10 +43,15 @@ public void flush() { } @Override - protected void injectFaultBeforeRead(ReaderGroupPruner pruner) throws Exception { - // Stop pruner (but do not close it). This will also stop the membership synchronizer. - pruner.stopAsync(); - latch.await(); - throw new Exception(); + protected void injectFault(ReaderGroupPruner pruner) throws Exception { + if (!latch.isReleased()) { + log.warn("injectFault: BEGIN"); + // Pause pruner (but do not close it). This will also pause the membership synchronizer. + pruner.pause(); + // Halt this processor thread until the latch is released. + latch.await(); + pruner.unpause(); + log.warn("injectFault: END"); + } } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 9ff1c0d2..a7f5f332 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -122,7 +122,7 @@ public void basicTest() throws Exception { final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); // Start initial set of processors. This will also create the necessary streams. - workerProcessGroup.start(0, 1, 2); + workerProcessGroup.start(0); // Prepare generator writer that will write to the stream read by the processor. @Cleanup @@ -150,7 +150,7 @@ public void basicTest() throws Exception { validationReaderConfig); EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, 30000); - final TestEventGenerator generator = new TestEventGenerator(6); + final TestEventGenerator generator = new TestEventGenerator(1); final TestEventValidator validator = new TestEventValidator(generator); // Write events to input stream. @@ -167,7 +167,8 @@ public void basicTest() throws Exception { log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); workerProcessGroup.start(3); - workerProcessGroup.stop(0, 1, 2); +// workerProcessGroup.stop(0); + workerProcessGroup.pause(0); Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); validator.validate(readerIterator); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 84b95c30..b0f25d34 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -30,6 +30,18 @@ public void start(int... instanceIds) { }); } + /** + * Processors are guaranteed to not process events written after this method returns. + */ + public void pause(int... instanceIds) { + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + workers.get(instanceId).pause(); + }); + } + + /** + * Processors are guaranteed to not process events after this method returns. + */ public void stop(int... instanceIds) { IntStream.of(instanceIds).forEach(instanceId -> { workers.get(instanceId).stopAsync(); From f43a2cd3108ad642846c44fd38d5d4e725aeeee7 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Wed, 8 Jul 2020 05:34:38 +0000 Subject: [PATCH 38/78] Basic pause of worker working Signed-off-by: Claudio Fahey --- .../example/streamprocessing/MembershipSynchronizer.java | 8 ++++++++ .../example/streamprocessing/ReaderGroupPruner.java | 3 ++- .../example/streamprocessing/TestEventValidator.java | 3 ++- .../example/streamprocessing/WorkerProcessGroup.java | 6 +++++- 4 files changed, 17 insertions(+), 3 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 8699a751..227d6bcf 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -279,4 +279,12 @@ protected void doStop() { task.cancel(false); } + public void pause() { + log.warn("paused"); + task.cancel(false); + } + + public void unpause() { + log.warn("unpause"); + } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 2fe9745c..f5c67f2f 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -96,11 +96,12 @@ protected void doStop() { public void pause() { log.warn("paused"); task.cancel(false); + membershipSynchronizer.pause(); } public void unpause() { log.warn("unpause"); -// throw new UnsupportedOperationException(); + membershipSynchronizer.unpause(); } @Override diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index 59f63378..addf092b 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -38,7 +38,8 @@ public void validate(Iterator events) { final long lastReceivedSequenceNumber = receivedSequenceNumbers.getOrDefault(event.key, -1L); log.info("event={}, lastReceivedSequenceNumber={}", event, lastReceivedSequenceNumber); if (event.sequenceNumber <= lastReceivedSequenceNumber) { - throw new IllegalStateException("Duplicate event"); + log.warn("Duplicate event; event={}, lastReceivedSequenceNumber={}", event, lastReceivedSequenceNumber); +// throw new IllegalStateException("Duplicate event"); } else if (event.sequenceNumber > lastReceivedSequenceNumber + 1) { throw new IllegalStateException("Gap"); } else { diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index b0f25d34..9b190155 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -47,7 +47,11 @@ public void stop(int... instanceIds) { workers.get(instanceId).stopAsync(); }); IntStream.of(instanceIds).forEach(instanceId -> { - workers.get(instanceId).awaitTerminated(); + try { + workers.get(instanceId).awaitTerminated(); + } catch (Exception e) { + log.warn("stop", e); + } workers.remove(instanceId); }); } From 5f9aeb09501c4fa4f82a312708ef4416727a2454 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 00:48:32 +0000 Subject: [PATCH 39/78] Use TestContext Signed-off-by: Claudio Fahey --- .../StreamProcessingTest.java | 109 ++++++++++++------ 1 file changed, 75 insertions(+), 34 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index a7f5f332..6eea2bb3 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -15,7 +15,9 @@ import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; +import lombok.Builder; import lombok.Cleanup; +import lombok.RequiredArgsConstructor; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -23,7 +25,9 @@ import org.slf4j.LoggerFactory; import java.util.UUID; +import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; public class StreamProcessingTest { static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); @@ -43,7 +47,8 @@ public static void tearDown() throws Exception { @Test public void noProcessorTest() throws Exception { - final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); + final String methodName = (new Object() { + }).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); // Create stream. @@ -51,14 +56,13 @@ public void noProcessorTest() throws Exception { final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); final String inputStreamName = "stream-" + UUID.randomUUID().toString(); SETUP_UTILS.get().createTestStream(inputStreamName, 6); - @Cleanup - final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); // Prepare writer that will write to the stream. - final Serializer serializer = new JSONSerializer<>(new TypeToken(){}.getType()); + final Serializer serializer = new JSONSerializer<>(new TypeToken() { + }.getType()); final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); - @Cleanup - final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); + @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); // Prepare reader that will read from the stream. final String outputStreamName = inputStreamName; @@ -68,14 +72,13 @@ public void noProcessorTest() throws Exception { final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(SETUP_UTILS.get().getStream(outputStreamName)) .build(); - @Cleanup - final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); + @Cleanup final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - @Cleanup - final EventStreamReader reader = clientFactory.createReader( + @Cleanup final EventStreamReader reader = clientFactory.createReader( readerId, readerGroup, - new JSONSerializer<>(new TypeToken(){}.getType()), + new JSONSerializer<>(new TypeToken() { + }.getType()), readerConfig); EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); @@ -94,8 +97,23 @@ public void noProcessorTest() throws Exception { log.info("SUCCESS"); } - @Test - public void basicTest() throws Exception { + @RequiredArgsConstructor + static class TestContext { + final EventStreamWriter writer; + final EventStreamReaderIterator readerIterator; + final TestEventGenerator generator; + final TestEventValidator validator; + final WorkerProcessGroup workerProcessGroup; + } + + void writeEventsAndValidate(TestContext ctx, int numEvents) { + // Write events to input stream. + Iterators.limit(ctx.generator, numEvents).forEachRemaining(event -> ctx.writer.writeEvent(Integer.toString(event.key), event)); + // Read events from output stream. Return when complete or throw exception if out of order or timeout. + ctx.validator.validate(ctx.readerIterator); + } + + private void test1(Consumer fun) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); @@ -153,27 +171,32 @@ public void basicTest() throws Exception { final TestEventGenerator generator = new TestEventGenerator(1); final TestEventValidator validator = new TestEventValidator(generator); - // Write events to input stream. - Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - // Read events from output stream. Return when complete or throw exception if out of order or timeout. - validator.validate(readerIterator); - - // Write and read additional events. - Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - validator.validate(readerIterator); - Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - validator.validate(readerIterator); - - log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); - - workerProcessGroup.start(3); -// workerProcessGroup.stop(0); - workerProcessGroup.pause(0); - - Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - validator.validate(readerIterator); - - log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); + final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup); + fun.accept(ctx); + +// // Write events to input stream. +// Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); +// // Read events from output stream. Return when complete or throw exception if out of order or timeout. +// validator.validate(readerIterator); +// +// // Write and read additional events. +// Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); +// validator.validate(readerIterator); +// Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); +// validator.validate(readerIterator); +// +// log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); +// +// workerProcessGroup.start(3); +//// workerProcessGroup.stop(0); +// workerProcessGroup.pause(0); +// +// Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); +// validator.validate(readerIterator); +// +// log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); +// +// fun.run(writer, readerIterator, generator, validator, workerProcessGroup); // Cleanup log.info("Cleanup"); @@ -189,4 +212,22 @@ public void basicTest() throws Exception { streamManager.deleteStream(scope, membershipSynchronizerStreamName); log.info("SUCCESS"); } + + @Test + public void basicTest() throws Exception { + final Consumer fun = ctx -> { + writeEventsAndValidate(ctx, 13); + writeEventsAndValidate(ctx, 3); + writeEventsAndValidate(ctx, 15); + log.info("getEventCountByInstanceId={}", ctx.validator.getEventCountByInstanceId()); + ctx.workerProcessGroup.start(3); +// ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.pause(0); + writeEventsAndValidate(ctx, 10); + log.info("getEventCountByInstanceId={}", ctx.validator.getEventCountByInstanceId()); + }; + test1(fun); + } + + } From f2bdf1ace25b0e63b6d03fed33d286e8cd455387 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 03:22:15 +0000 Subject: [PATCH 40/78] Now validating that stopped instances do not process events Signed-off-by: Claudio Fahey --- .../StreamProcessingTest.java | 123 +++++++++++------- .../streamprocessing/TestEventGenerator.java | 1 - .../streamprocessing/TestEventValidator.java | 4 + 3 files changed, 77 insertions(+), 51 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 6eea2bb3..988f5669 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -1,6 +1,7 @@ package io.pravega.example.streamprocessing; import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; import com.google.gson.reflect.TypeToken; import io.pravega.client.ClientConfig; import io.pravega.client.EventStreamClientFactory; @@ -15,19 +16,25 @@ import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; -import lombok.Builder; import lombok.Cleanup; import lombok.RequiredArgsConstructor; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.text.MessageFormat; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.UUID; -import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class StreamProcessingTest { static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); @@ -106,14 +113,28 @@ static class TestContext { final WorkerProcessGroup workerProcessGroup; } - void writeEventsAndValidate(TestContext ctx, int numEvents) { + void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { + ctx.validator.clearCounters(); // Write events to input stream. Iterators.limit(ctx.generator, numEvents).forEachRemaining(event -> ctx.writer.writeEvent(Integer.toString(event.key), event)); // Read events from output stream. Return when complete or throw exception if out of order or timeout. ctx.validator.validate(ctx.readerIterator); + // Confirm that only instances in expectedInstanceIds have processed the events. + final Map eventCountByInstanceId = ctx.validator.getEventCountByInstanceId(); + final Set actualInstanceIds = eventCountByInstanceId.keySet(); + final Set expectedInstanceIdsSet = Arrays.stream(expectedInstanceIds).boxed().collect(Collectors.toCollection(HashSet::new)); + log.info("writeEventsAndValidate: eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); + Assert.assertTrue(MessageFormat.format("eventCountByInstanceId={0}, expectedInstanceIdsSet={1}", eventCountByInstanceId, expectedInstanceIdsSet), + Sets.difference(actualInstanceIds, expectedInstanceIdsSet).isEmpty()); + // Warn if any instances are idle. This cannot be an assertion because this may happen under normal conditions. + final Sets.SetView idleInstanceIds = Sets.difference(expectedInstanceIdsSet, actualInstanceIds); + if (!idleInstanceIds.isEmpty()) { + log.warn("writeEventsAndValidate: Some instances processed no events; eventCountByInstanceId={}, expectedInstanceIdsSet={}", + eventCountByInstanceId, expectedInstanceIdsSet); + } } - private void test1(Consumer fun) throws Exception { + private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, Consumer func) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); @@ -134,13 +155,13 @@ private void test1(Consumer fun) throws Exception { .inputStreamName(inputStreamName) .outputStreamName(outputStreamName) .membershipSynchronizerStreamName(membershipSynchronizerStreamName) - .numSegments(6) + .numSegments(numSegments) .build(); @Cleanup final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); // Start initial set of processors. This will also create the necessary streams. - workerProcessGroup.start(0); + workerProcessGroup.start(IntStream.range(0, numInitialInstances).toArray()); // Prepare generator writer that will write to the stream read by the processor. @Cleanup @@ -166,39 +187,13 @@ private void test1(Consumer fun) throws Exception { validationReaderGroupName, new JSONSerializer<>(new TypeToken(){}.getType()), validationReaderConfig); - EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, 30000); - - final TestEventGenerator generator = new TestEventGenerator(1); + final long readTimeoutMills = 60000; + EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, readTimeoutMills); + final TestEventGenerator generator = new TestEventGenerator(numKeys); final TestEventValidator validator = new TestEventValidator(generator); - final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup); - fun.accept(ctx); - -// // Write events to input stream. -// Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); -// // Read events from output stream. Return when complete or throw exception if out of order or timeout. -// validator.validate(readerIterator); -// -// // Write and read additional events. -// Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); -// validator.validate(readerIterator); -// Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); -// validator.validate(readerIterator); -// -// log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); -// -// workerProcessGroup.start(3); -//// workerProcessGroup.stop(0); -// workerProcessGroup.pause(0); -// -// Iterators.limit(generator, 10).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); -// validator.validate(readerIterator); -// -// log.info("getEventCountByInstanceId={}", validator.getEventCountByInstanceId()); -// -// fun.run(writer, readerIterator, generator, validator, workerProcessGroup); - - // Cleanup + func.accept(ctx); + log.info("Cleanup"); workerProcessGroup.close(); validationReader.close(); @@ -210,24 +205,52 @@ private void test1(Consumer fun) throws Exception { streamManager.deleteStream(scope, inputStreamName); streamManager.deleteStream(scope, outputStreamName); streamManager.deleteStream(scope, membershipSynchronizerStreamName); - log.info("SUCCESS"); } @Test - public void basicTest() throws Exception { - final Consumer fun = ctx -> { - writeEventsAndValidate(ctx, 13); - writeEventsAndValidate(ctx, 3); - writeEventsAndValidate(ctx, 15); - log.info("getEventCountByInstanceId={}", ctx.validator.getEventCountByInstanceId()); - ctx.workerProcessGroup.start(3); -// ctx.workerProcessGroup.stop(0); + public void trivialTest() throws Exception { + endToEndTest(1, 1, 1, ctx -> { + writeEventsAndValidate(ctx, 20, new int[]{0}); + }); + } + + @Test + public void gracefulRestart1of1Test() throws Exception { + endToEndTest(6, 24, 1, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + }); + } + + @Test + public void gracefulStop1of2Test() throws Exception { + endToEndTest(6, 24, 2, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0, 1}); + ctx.workerProcessGroup.stop(0); + writeEventsAndValidate(ctx, 90, new int[]{1}); + }); + } + + @Test + public void killAndRestart1of1Test() throws Exception { + endToEndTest(6, 24, 1, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); ctx.workerProcessGroup.pause(0); - writeEventsAndValidate(ctx, 10); - log.info("getEventCountByInstanceId={}", ctx.validator.getEventCountByInstanceId()); - }; - test1(fun); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + }); } + @Test + public void kill5of6Test() throws Exception { + endToEndTest(6, 24, 6, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); + ctx.workerProcessGroup.pause(0, 1, 2, 3, 4); + writeEventsAndValidate(ctx, 90, new int[]{5}); + }); + } + // TODO: pause and resume } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java index 541eca89..bae3def5 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java @@ -19,7 +19,6 @@ public TestEventGenerator(int numKeys) { this.numKeys = numKeys; this.lastKey = numKeys - 1; lastSequenceNumbers = new HashMap<>(); -// IntStream.range(0, numKeys).forEach(key -> lastSequenceNumbers.put(key, 0L)); } @Override diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index addf092b..69576f32 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -58,6 +58,10 @@ public void validate(Iterator events) { throw new IllegalStateException("No more events"); } + public void clearCounters() { + eventCountByInstanceId.clear(); + } + public Map getEventCountByInstanceId() { return eventCountByInstanceId; } From 4601f08c0b4150333fa78143e2455f0428af3b2e Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 05:32:56 +0000 Subject: [PATCH 41/78] Add license headers Signed-off-by: Claudio Fahey --- .../streamprocessing/WorkerProcess.java | 10 +++ .../streamprocessing/WorkerProcessConfig.java | 10 +++ .../streamprocessing/WorkerProcessGroup.java | 10 +++ .../utils/EventStreamReaderIterator.java | 10 +++ .../java/io/pravega/utils/SetupUtils.java | 70 +++++++++---------- 5 files changed, 72 insertions(+), 38 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index 9818ab99..ce5be9bf 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import com.google.common.util.concurrent.AbstractExecutionThreadService; diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java index ebbeb04a..182ab474 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import io.pravega.client.ClientConfig; diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 9b190155..7e155ff4 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import lombok.Builder; diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java index ee130153..72001457 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.utils; import io.pravega.client.stream.EventRead; diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java index 5ba1fbac..bdff1eb7 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java @@ -1,51 +1,45 @@ /* - * Copyright 2019 Flavio Junqueira + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed 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 + * 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 io.pravega.utils; - import com.google.common.base.Preconditions; - import io.pravega.client.ClientConfig; - import io.pravega.client.EventStreamClientFactory; - import io.pravega.client.admin.StreamManager; - import io.pravega.client.stream.ScalingPolicy; - import io.pravega.client.stream.Stream; - import io.pravega.client.stream.StreamConfiguration; - import io.pravega.client.stream.impl.Controller; - import io.pravega.client.stream.impl.ControllerImpl; - import io.pravega.client.stream.impl.ControllerImplConfig; - import io.pravega.common.concurrent.ExecutorServiceHelpers; - import io.pravega.local.InProcPravegaCluster; - import lombok.Cleanup; - import lombok.Getter; - import lombok.Setter; - import org.apache.commons.lang3.RandomStringUtils; - import org.slf4j.Logger; - import org.slf4j.LoggerFactory; - - import javax.annotation.concurrent.NotThreadSafe; - import java.io.IOException; - import java.io.InputStream; - import java.net.ServerSocket; - import java.net.URI; - import java.nio.file.Files; - import java.nio.file.Path; - import java.nio.file.StandardCopyOption; - import java.util.concurrent.ScheduledExecutorService; - import java.util.concurrent.atomic.AtomicBoolean; - import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.base.Preconditions; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.impl.Controller; +import io.pravega.client.stream.impl.ControllerImpl; +import io.pravega.client.stream.impl.ControllerImplConfig; +import io.pravega.common.concurrent.ExecutorServiceHelpers; +import io.pravega.local.InProcPravegaCluster; +import lombok.Cleanup; +import lombok.Getter; +import lombok.Setter; +import org.apache.commons.lang3.RandomStringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; +import java.io.IOException; +import java.io.InputStream; +import java.net.ServerSocket; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; /** * Utility functions for creating the test setup. From 5f465510a90433ae79e04b44d2570e26623f61f2 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 05:38:35 +0000 Subject: [PATCH 42/78] Add unit tests for TestEventValidator Signed-off-by: Claudio Fahey --- .../streamprocessing/EventValidatorTest.java | 30 ---- .../MissingEventException.java | 17 ++ .../NoMoreEventsException.java | 17 ++ .../StreamProcessingTest.java | 94 +++++----- .../example/streamprocessing/TestEvent.java | 8 +- .../streamprocessing/TestEventGenerator.java | 12 +- .../streamprocessing/TestEventValidator.java | 39 +++-- .../TestEventValidatorUnitTests.java | 161 ++++++++++++++++++ 8 files changed, 284 insertions(+), 94 deletions(-) delete mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/MissingEventException.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/NoMoreEventsException.java create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidatorUnitTests.java diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java deleted file mode 100644 index c55d3996..00000000 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/EventValidatorTest.java +++ /dev/null @@ -1,30 +0,0 @@ -package io.pravega.example.streamprocessing; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import io.pravega.test.common.AssertExtensions; - -import java.util.Iterator; - -public class EventValidatorTest { - static final Logger log = LoggerFactory.getLogger(EventValidatorTest.class); - - @Test - public void basicTest() throws Exception { - TestEventGenerator generator = new TestEventGenerator(6); - final Iterator generated = ImmutableList.copyOf(Iterators.limit(generator, 100)).iterator(); - TestEventValidator validator = new TestEventValidator(generator); - validator.validate(generated); - } - - @Test - public void missingEventTest() throws Exception { - TestEventGenerator generator = new TestEventGenerator(6); - final Iterator generated = ImmutableList.copyOf(Iterators.limit(generator, 10)).iterator(); - TestEventValidator validator = new TestEventValidator(generator); - AssertExtensions.assertThrows(Exception.class, () -> validator.validate(Iterators.limit(generated, 9))); - } -} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/MissingEventException.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/MissingEventException.java new file mode 100644 index 00000000..bcc46a48 --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/MissingEventException.java @@ -0,0 +1,17 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +class MissingEventException extends RuntimeException { + public MissingEventException(String s) { + super(s); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/NoMoreEventsException.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/NoMoreEventsException.java new file mode 100644 index 00000000..b3c863cc --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/NoMoreEventsException.java @@ -0,0 +1,17 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +class NoMoreEventsException extends RuntimeException { + public NoMoreEventsException(String s) { + super(s); + } +} diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 988f5669..fa5deece 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import com.google.common.collect.Iterators; @@ -52,6 +62,36 @@ public static void tearDown() throws Exception { SETUP_UTILS.get().stopAllServices(); } + @RequiredArgsConstructor + static class TestContext { + final EventStreamWriter writer; + final EventStreamReaderIterator readerIterator; + final TestEventGenerator generator; + final TestEventValidator validator; + final WorkerProcessGroup workerProcessGroup; + } + + void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { + ctx.validator.clearCounters(); + // Write events to input stream. + Iterators.limit(ctx.generator, numEvents).forEachRemaining(event -> ctx.writer.writeEvent(Integer.toString(event.key), event)); + // Read events from output stream. Return when complete or throw exception if out of order or timeout. + ctx.validator.validate(ctx.readerIterator, ctx.generator.getLastSequenceNumbers()); + // Confirm that only instances in expectedInstanceIds have processed the events. + final Map eventCountByInstanceId = ctx.validator.getEventCountByInstanceId(); + final Set actualInstanceIds = eventCountByInstanceId.keySet(); + final Set expectedInstanceIdsSet = Arrays.stream(expectedInstanceIds).boxed().collect(Collectors.toCollection(HashSet::new)); + log.info("writeEventsAndValidate: eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); + Assert.assertTrue(MessageFormat.format("eventCountByInstanceId={0}, expectedInstanceIdsSet={1}", eventCountByInstanceId, expectedInstanceIdsSet), + Sets.difference(actualInstanceIds, expectedInstanceIdsSet).isEmpty()); + // Warn if any instances are idle. This cannot be an assertion because this may happen under normal conditions. + final Sets.SetView idleInstanceIds = Sets.difference(expectedInstanceIdsSet, actualInstanceIds); + if (!idleInstanceIds.isEmpty()) { + log.warn("writeEventsAndValidate: Some instances processed no events; eventCountByInstanceId={}, expectedInstanceIdsSet={}", + eventCountByInstanceId, expectedInstanceIdsSet); + } + } + @Test public void noProcessorTest() throws Exception { final String methodName = (new Object() { @@ -66,8 +106,7 @@ public void noProcessorTest() throws Exception { @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); // Prepare writer that will write to the stream. - final Serializer serializer = new JSONSerializer<>(new TypeToken() { - }.getType()); + final Serializer serializer = new JSONSerializer<>(new TypeToken() {}.getType()); final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); @@ -84,54 +123,17 @@ public void noProcessorTest() throws Exception { @Cleanup final EventStreamReader reader = clientFactory.createReader( readerId, readerGroup, - new JSONSerializer<>(new TypeToken() { - }.getType()), + new JSONSerializer<>(new TypeToken() {}.getType()), readerConfig); EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(reader, 30000); - // Create event generator instance. final TestEventGenerator generator = new TestEventGenerator(6); - // Create event validator instance. - final TestEventValidator validator = new TestEventValidator(generator); - // Write historical events. - Iterators.limit(generator, 13).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - // Read events from output stream. Return when complete or throw exception if out of order or timeout. - validator.validate(readerIterator); - Iterators.limit(generator, 3).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - validator.validate(readerIterator); - Iterators.limit(generator, 15).forEachRemaining(event -> writer.writeEvent(Integer.toString(event.key), event)); - validator.validate(readerIterator); - log.info("SUCCESS"); - } + final TestEventValidator validator = new TestEventValidator(); + final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, null); - @RequiredArgsConstructor - static class TestContext { - final EventStreamWriter writer; - final EventStreamReaderIterator readerIterator; - final TestEventGenerator generator; - final TestEventValidator validator; - final WorkerProcessGroup workerProcessGroup; - } - - void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { - ctx.validator.clearCounters(); - // Write events to input stream. - Iterators.limit(ctx.generator, numEvents).forEachRemaining(event -> ctx.writer.writeEvent(Integer.toString(event.key), event)); - // Read events from output stream. Return when complete or throw exception if out of order or timeout. - ctx.validator.validate(ctx.readerIterator); - // Confirm that only instances in expectedInstanceIds have processed the events. - final Map eventCountByInstanceId = ctx.validator.getEventCountByInstanceId(); - final Set actualInstanceIds = eventCountByInstanceId.keySet(); - final Set expectedInstanceIdsSet = Arrays.stream(expectedInstanceIds).boxed().collect(Collectors.toCollection(HashSet::new)); - log.info("writeEventsAndValidate: eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); - Assert.assertTrue(MessageFormat.format("eventCountByInstanceId={0}, expectedInstanceIdsSet={1}", eventCountByInstanceId, expectedInstanceIdsSet), - Sets.difference(actualInstanceIds, expectedInstanceIdsSet).isEmpty()); - // Warn if any instances are idle. This cannot be an assertion because this may happen under normal conditions. - final Sets.SetView idleInstanceIds = Sets.difference(expectedInstanceIdsSet, actualInstanceIds); - if (!idleInstanceIds.isEmpty()) { - log.warn("writeEventsAndValidate: Some instances processed no events; eventCountByInstanceId={}, expectedInstanceIdsSet={}", - eventCountByInstanceId, expectedInstanceIdsSet); - } + writeEventsAndValidate(ctx, 13, new int[]{-1}); + writeEventsAndValidate(ctx, 3, new int[]{-1}); + writeEventsAndValidate(ctx, 15, new int[]{-1}); } private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, Consumer func) throws Exception { @@ -190,7 +192,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, final long readTimeoutMills = 60000; EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, readTimeoutMills); final TestEventGenerator generator = new TestEventGenerator(numKeys); - final TestEventValidator validator = new TestEventValidator(generator); + final TestEventValidator validator = new TestEventValidator(); final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup); func.accept(ctx); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java index 3bcd35d1..cd41b4cf 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEvent.java @@ -18,7 +18,13 @@ public class TestEvent { public TestEvent(int key, long sequenceNumber) { this.key = key; this.sequenceNumber = sequenceNumber; - processedByInstanceId = -1; + this.processedByInstanceId = -1; + } + + public TestEvent(int key, long sequenceNumber, int processedByInstanceId) { + this.key = key; + this.sequenceNumber = sequenceNumber; + this.processedByInstanceId = processedByInstanceId; } @Override diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java index bae3def5..c032af5e 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import org.slf4j.Logger; @@ -29,7 +39,7 @@ public boolean hasNext() { @Override public TestEvent next() { lastKey = (lastKey + 1) % numKeys;; - final Long sequenceNumber = lastSequenceNumbers.getOrDefault(lastKey, -1L) + 1; + final long sequenceNumber = lastSequenceNumbers.getOrDefault(lastKey, -1L) + 1; lastSequenceNumbers.put(lastKey, sequenceNumber); final TestEvent event = new TestEvent(lastKey, sequenceNumber); log.info("event={}", event); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index 69576f32..f43fb275 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -1,3 +1,13 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ package io.pravega.example.streamprocessing; import org.slf4j.Logger; @@ -10,24 +20,17 @@ public class TestEventValidator { static final Logger log = LoggerFactory.getLogger(TestEventValidator.class); - private final TestEventGenerator generator; - // map from routing key to sequence number + // Map from routing key to highest received sequence number. private final Map receivedSequenceNumbers = new HashMap<>(); - - // map from instanceId to count of events + // Map from instanceId to count of events processed by the instance, excluding duplicates. private final Map eventCountByInstanceId = new HashMap<>(); + private long duplicateEventCount; - public TestEventValidator(TestEventGenerator generator) { - this.generator = generator; - } - - public void validate(Iterator events) { + public void validate(Iterator events, Map expectedLastSequenceNumbers) { // pendingSequenceNumbers contains a map from key to sequence number for events that have been generated but not yet received by validate. // A key is removed when all events up to the generated sequence number for that key are received. final Map pendingSequenceNumbers = new HashMap<>(); - final Map generatedSequenceNumbers = generator.getLastSequenceNumbers(); - log.info("generatedSequenceNumbers={}", generatedSequenceNumbers); - generatedSequenceNumbers.forEach((key, sequenceNumber) -> { + expectedLastSequenceNumbers.forEach((key, sequenceNumber) -> { if (receivedSequenceNumbers.getOrDefault(key, -1L) < sequenceNumber) { pendingSequenceNumbers.put(key, sequenceNumber); } @@ -39,30 +42,34 @@ public void validate(Iterator events) { log.info("event={}, lastReceivedSequenceNumber={}", event, lastReceivedSequenceNumber); if (event.sequenceNumber <= lastReceivedSequenceNumber) { log.warn("Duplicate event; event={}, lastReceivedSequenceNumber={}", event, lastReceivedSequenceNumber); -// throw new IllegalStateException("Duplicate event"); + duplicateEventCount++; } else if (event.sequenceNumber > lastReceivedSequenceNumber + 1) { - throw new IllegalStateException("Gap"); + throw new MissingEventException("Detected missing event"); } else { receivedSequenceNumbers.put(event.key, event.sequenceNumber); eventCountByInstanceId.merge(event.processedByInstanceId, 1L, Long::sum); // increment counter if (pendingSequenceNumbers.getOrDefault(event.key, -1L) <= event.sequenceNumber) { pendingSequenceNumbers.remove(event.key); if (pendingSequenceNumbers.size() == 0) { - // All data received. log.info("All data received; receivedSequenceNumbers={}", receivedSequenceNumbers); return; } } } } - throw new IllegalStateException("No more events"); + throw new NoMoreEventsException("No more events but all expected events were not received"); } public void clearCounters() { eventCountByInstanceId.clear(); + duplicateEventCount = 0; } public Map getEventCountByInstanceId() { return eventCountByInstanceId; } + + public long getDuplicateEventCount() { + return duplicateEventCount; + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidatorUnitTests.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidatorUnitTests.java new file mode 100644 index 00000000..6ff6fbfb --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidatorUnitTests.java @@ -0,0 +1,161 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import com.google.common.collect.ImmutableMap; +import io.pravega.test.common.AssertExtensions; +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * Unit tests for TestEventValidator. + */ +public class TestEventValidatorUnitTests { + static final Logger log = LoggerFactory.getLogger(TestEventValidatorUnitTests.class); + + @Test + public void exactlyOnceTest() { + List events = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), + new TestEvent(0, 1L, 10), + new TestEvent(0, 2L, 11), + new TestEvent(0, 3L, 11), + new TestEvent(1, 2L, 11)); + Map expectedLastSequenceNumbers = ImmutableMap.of( + 0, 3L, + 1, 2L); + Map expectedEventCountByInstanceId = ImmutableMap.of( + 10, 4L, + 11, 3L); + TestEventValidator validator = new TestEventValidator(); + validator.validate(events.iterator(), expectedLastSequenceNumbers); + Assert.assertEquals(expectedEventCountByInstanceId, validator.getEventCountByInstanceId()); + Assert.assertEquals(0, validator.getDuplicateEventCount()); + } + + @Test + public void duplicateEventTest() { + List events = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), + new TestEvent(0, 0L, 11), // duplicate + new TestEvent(0, 1L, 11), + new TestEvent(0, 2L, 11), + new TestEvent(0, 3L, 11), + new TestEvent(1, 2L, 11)); + Map expectedLastSequenceNumbers = ImmutableMap.of( + 0, 3L, + 1, 2L); + Map expectedEventCountByInstanceId = ImmutableMap.of( + 10, 3L, + 11, 4L); + TestEventValidator validator = new TestEventValidator(); + validator.validate(events.iterator(), expectedLastSequenceNumbers); + Assert.assertEquals(expectedEventCountByInstanceId, validator.getEventCountByInstanceId()); + Assert.assertEquals(1, validator.getDuplicateEventCount()); + } + + @Test + public void rewindTest() { + List events = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), + new TestEvent(1, 0L, 11), // rewind, duplicate + new TestEvent(1, 1L, 11), // duplicate + new TestEvent(0, 1L, 11), + new TestEvent(0, 2L, 11), + new TestEvent(0, 3L, 11), + new TestEvent(1, 2L, 11)); + Map expectedLastSequenceNumbers = ImmutableMap.of( + 0, 3L, + 1, 2L); + Map expectedEventCountByInstanceId = ImmutableMap.of( + 10, 3L, + 11, 4L); + TestEventValidator validator = new TestEventValidator(); + validator.validate(events.iterator(), expectedLastSequenceNumbers); + Assert.assertEquals(expectedEventCountByInstanceId, validator.getEventCountByInstanceId()); + Assert.assertEquals(2, validator.getDuplicateEventCount()); + } + + @Test + public void clearCountersTest() { + List events1 = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), + new TestEvent(1, 0L, 11), // rewind, duplicate + new TestEvent(1, 1L, 11), // duplicate + new TestEvent(0, 1L, 11), + new TestEvent(0, 2L, 11), + new TestEvent(0, 3L, 11), + new TestEvent(1, 2L, 11)); + Map expectedLastSequenceNumbers1 = ImmutableMap.of( + 0, 3L, + 1, 2L); + TestEventValidator validator = new TestEventValidator(); + validator.validate(events1.iterator(), expectedLastSequenceNumbers1); + validator.clearCounters(); + List events2 = Arrays.asList( + new TestEvent(100, 0L, 10), + new TestEvent(100, 0L, 10), // duplicate + new TestEvent(100, 1L, 10)); + Map expectedLastSequenceNumbers2 = ImmutableMap.of( + 100, 1L); + Map expectedEventCountByInstanceId2 = ImmutableMap.of( + 10, 2L); + validator.validate(events2.iterator(), expectedLastSequenceNumbers2); + Assert.assertEquals(expectedEventCountByInstanceId2, validator.getEventCountByInstanceId()); + Assert.assertEquals(1, validator.getDuplicateEventCount()); + } + + @Test + public void missingEventTest() { + List events = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), + new TestEvent(0, 2L, 11), // missing prior event + new TestEvent(0, 3L, 11), + new TestEvent(1, 2L, 11)); + Map expectedLastSequenceNumbers = ImmutableMap.of( + 0, 3L, + 1, 2L); + TestEventValidator validator = new TestEventValidator(); + AssertExtensions.assertThrows(MissingEventException.class, () -> validator.validate(events.iterator(), expectedLastSequenceNumbers)); + } + + @Test + public void missingFinalEventTest() { + List events = Arrays.asList( + new TestEvent(0, 0L, 10), + new TestEvent(1, 0L, 10), + new TestEvent(1, 1L, 10), // missing following event + new TestEvent(0, 1L, 10), + new TestEvent(0, 2L, 11), + new TestEvent(0, 3L, 11)); + Map expectedLastSequenceNumbers = ImmutableMap.of( + 0, 3L, + 1, 2L); + TestEventValidator validator = new TestEventValidator(); + AssertExtensions.assertThrows(NoMoreEventsException.class, () -> validator.validate(events.iterator(), expectedLastSequenceNumbers)); + } +} From da84a01ec3eab7e51e68d2b39c02a757e666eb14 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 06:15:24 +0000 Subject: [PATCH 43/78] Add testing documentation Signed-off-by: Claudio Fahey --- .../example/streamprocessing/Testing.md | 22 --------------- .../StreamProcessingTest.java | 27 +++++++++++++++++-- .../streamprocessing/TestEventGenerator.java | 5 ++++ .../streamprocessing/TestEventValidator.java | 4 +++ .../streamprocessing/WorkerProcess.java | 3 +++ .../streamprocessing/WorkerProcessGroup.java | 3 +++ .../utils/EventStreamReaderIterator.java | 3 +++ .../java/io/pravega/utils/SetupUtils.java | 4 +-- 8 files changed, 44 insertions(+), 27 deletions(-) delete mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md deleted file mode 100644 index c9d032f5..00000000 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/Testing.md +++ /dev/null @@ -1,22 +0,0 @@ - -# Ungraceful shutdown - -- Start multiple JVMs for processors -- Write events -- Read processed events -- Kill -9 JVMs -- Start new JVMs for processors -- Validate that all events are processed at least once. -- Validate that, if an idempotent writer were used (IdempotentEventWriter), all events would be processed exactly once. -- Ordering guarantee: maintains per-routing key ordering, with rewind. - -# Graceful shutdown - -- Same as above but use normal kill. -- Validate exactly once - -# Temporary network partition - -- Must block access from JVM to Pravega. -- Run in Docker and use iptables to block IPs? -- Must create Docker image with this application. \ No newline at end of file diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index fa5deece..9126341a 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -71,6 +71,14 @@ static class TestContext { final WorkerProcessGroup workerProcessGroup; } + /** + * Write the given number of events to the Pravega input stream. + * Then read events from the Pravega output stream to ensure that the processors produced the correct result. + * + * @param ctx provides access to the generator, writer, etc. + * @param numEvents number of events to write + * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. + */ void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { ctx.validator.clearCounters(); // Write events to input stream. @@ -92,6 +100,9 @@ void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstan } } + /** + * Write events to a Pravega stream, read events from the same stream, and validate expected ordering. + */ @Test public void noProcessorTest() throws Exception { final String methodName = (new Object() { @@ -136,6 +147,17 @@ public void noProcessorTest() throws Exception { writeEventsAndValidate(ctx, 15, new int[]{-1}); } + /** + * Write events to the input stream. Start multiple processors which can read from the input stream + * and write to the output stream. Validate events in the output stream. + * This method performs the setup and teardown. The provided function func performs the actual write and read + * of events; stops, pauses, and starts processor instances; and validates results. + * + * @param numSegments number of stream segments + * @param numKeys number of unique routing keys + * @param numInitialInstances number of initial processor instances + * @param func function to run to write and write events, etc. + */ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, Consumer func) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); @@ -213,6 +235,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, public void trivialTest() throws Exception { endToEndTest(1, 1, 1, ctx -> { writeEventsAndValidate(ctx, 20, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }); } @@ -223,6 +246,7 @@ public void gracefulRestart1of1Test() throws Exception { ctx.workerProcessGroup.stop(0); ctx.workerProcessGroup.start(1); writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }); } @@ -232,6 +256,7 @@ public void gracefulStop1of2Test() throws Exception { writeEventsAndValidate(ctx, 100, new int[]{0, 1}); ctx.workerProcessGroup.stop(0); writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }); } @@ -253,6 +278,4 @@ public void kill5of6Test() throws Exception { writeEventsAndValidate(ctx, 90, new int[]{5}); }); } - - // TODO: pause and resume } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java index c032af5e..26128644 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventGenerator.java @@ -17,6 +17,11 @@ import java.util.Iterator; import java.util.Map; +/** + * An Iterator that generates TestEvent elements. + * Events will have routing keys between 0 and numKeys - 1. + * For each routing key, sequence numbers will begin at 0 and increment by 1. + */ public class TestEventGenerator implements Iterator { static final Logger log = LoggerFactory.getLogger(TestEventGenerator.class); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index f43fb275..821e2369 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -17,6 +17,10 @@ import java.util.Iterator; import java.util.Map; +/** + * This validates that the sequence of events provided is consistent with the guarantees provided by the + * AtLeastOnceProcessor. + */ public class TestEventValidator { static final Logger log = LoggerFactory.getLogger(TestEventValidator.class); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index ce5be9bf..d56ba8c6 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -33,6 +33,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicReference; +/** + * This simulates an independent process running an AtLeastOnceProcessor. + */ @Builder public class WorkerProcess extends AbstractExecutionThreadService { private static final Logger log = LoggerFactory.getLogger(WorkerProcess.class); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 7e155ff4..9e846ccf 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -18,6 +18,9 @@ import java.util.Map; import java.util.stream.IntStream; +/** + * This manages a group of WorkerProcess instances. + */ @Builder public class WorkerProcessGroup implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(WorkerProcessGroup.class); diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java index 72001457..f974c102 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -16,6 +16,9 @@ import java.util.Iterator; import java.util.Optional; +/** + * An Iterator for reading from a Pravega stream. + */ public class EventStreamReaderIterator implements Iterator { private final EventStreamReader reader; private final long timeoutMillis; diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java index bdff1eb7..9318e7d5 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java @@ -42,12 +42,10 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * Utility functions for creating the test setup. + * Utility functions for creating an in-process Pravega server or connecting to an external Pravega server. */ -//@Slf4j @NotThreadSafe public final class SetupUtils { - static Logger log = LoggerFactory.getLogger(SetupUtils.class); private static final ScheduledExecutorService DEFAULT_SCHEDULED_EXECUTOR_SERVICE = ExecutorServiceHelpers.newScheduledThreadPool(3, "SetupUtils"); From d409f440e7f1c5fbe13ecf049c5ad4d8c4481954 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 17:51:37 +0000 Subject: [PATCH 44/78] Add test forcing duplicates. Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessorInstrumented.java | 14 +++++ .../StreamProcessingTest.java | 61 ++++++++++++++++++- .../streamprocessing/WorkerProcess.java | 10 ++- .../streamprocessing/WorkerProcessGroup.java | 6 ++ 4 files changed, 86 insertions(+), 5 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index e7097ba6..0621ddf3 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -7,6 +7,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor { @@ -15,6 +17,8 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writer; private final ReusableLatch latch; + private final AtomicLong unflushedEventCount = new AtomicLong(0); + private final AtomicBoolean preventFlushFlag = new AtomicBoolean(false); public AtLeastOnceProcessorInstrumented( Supplier pruner, @@ -35,11 +39,17 @@ public void process(EventRead eventRead) { event.processedByInstanceId = instanceId; log.info("process: event={}", event); writer.writeEvent(Integer.toString(event.key), event); + unflushedEventCount.incrementAndGet(); } @Override public void flush() { + if (preventFlushFlag.get()) { + throw new RuntimeException("Flush called but this test requires that that flush not be called. Try to rerun the test."); + } writer.flush(); + final long flushedEventCount = unflushedEventCount.getAndSet(0); + log.info("flush: Flushed {} events", flushedEventCount); } @Override @@ -54,4 +64,8 @@ protected void injectFault(ReaderGroupPruner pruner) throws Exception { log.warn("injectFault: END"); } } + + public void preventFlush() { + preventFlushFlag.set(true); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 9126341a..1b4bd97e 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -69,6 +69,7 @@ static class TestContext { final TestEventGenerator generator; final TestEventValidator validator; final WorkerProcessGroup workerProcessGroup; + final long checkpointPeriodMs; } /** @@ -140,7 +141,7 @@ public void noProcessorTest() throws Exception { final TestEventGenerator generator = new TestEventGenerator(6); final TestEventValidator validator = new TestEventValidator(); - final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, null); + final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, null, 0); writeEventsAndValidate(ctx, 13, new int[]{-1}); writeEventsAndValidate(ctx, 3, new int[]{-1}); @@ -168,6 +169,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, final String outputStreamName = "output-stream-" + UUID.randomUUID().toString(); final String membershipSynchronizerStreamName = "ms-" + UUID.randomUUID().toString(); final String inputStreamReaderGroupName = "rg" + UUID.randomUUID().toString().replace("-", ""); + final long checkpointPeriodMs = 1000; @Cleanup StreamManager streamManager = StreamManager.create(clientConfig); @@ -180,6 +182,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, .outputStreamName(outputStreamName) .membershipSynchronizerStreamName(membershipSynchronizerStreamName) .numSegments(numSegments) + .checkpointPeriodMs(checkpointPeriodMs) .build(); @Cleanup final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); @@ -215,7 +218,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, readTimeoutMills); final TestEventGenerator generator = new TestEventGenerator(numKeys); final TestEventValidator validator = new TestEventValidator(); - final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup); + final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup, checkpointPeriodMs); func.accept(ctx); log.info("Cleanup"); @@ -270,6 +273,60 @@ public void killAndRestart1of1Test() throws Exception { }); } + @Test + public void killAndRestart1of1WhenIdleTest() throws Exception { + endToEndTest(6, 24, 1, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + try { + Thread.sleep(2*ctx.checkpointPeriodMs); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + // Kill the worker instance. + ctx.workerProcessGroup.pause(0); + // Start a new worker instance. It should identify the dead worker and call readerOffline(null). + // The new worker should resume exactly where the killed worker left off, producing no duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 19, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }); + } + + @Test + public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { + endToEndTest(6, 24, 1, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + try { + Thread.sleep(2*ctx.checkpointPeriodMs); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. + // If this happens, we will throw an exception and this test will fail. This should be rare. + ctx.workerProcessGroup.preventFlush(0); + // Write some events that will be processed but not flushed. + final int expectedDuplicateEventCount = 3; + writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Kill the worker instance. + ctx.workerProcessGroup.pause(0); + // Start a new worker instance. It should identify the dead worker and call readerOffline(null). + // The new worker should produce duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 3, new int[]{1}); + Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + }); + } + + @Test public void kill5of6Test() throws Exception { endToEndTest(6, 24, 6, ctx -> { diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index d56ba8c6..f565a46d 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -44,7 +44,7 @@ public class WorkerProcess extends AbstractExecutionThreadService { private final int instanceId; private final ReusableLatch latch = new ReusableLatch(true); - private final AtomicReference> processor = new AtomicReference<>(); + private final AtomicReference processor = new AtomicReference<>(); // Create the input, output, and state synchronizer streams (ignored if they already exist). public void init() { @@ -85,7 +85,7 @@ protected void run() throws Exception { serializer, EventWriterConfig.builder().build())) { - final AtLeastOnceProcessor proc = new AtLeastOnceProcessorInstrumented( + final AtLeastOnceProcessorInstrumented proc = new AtLeastOnceProcessorInstrumented( () -> ReaderGroupPruner.create( readerGroup, config.membershipSynchronizerStreamName, @@ -112,7 +112,7 @@ protected void run() throws Exception { @Override protected void triggerShutdown() { log.info("triggerShutdown: BEGIN"); - final AtLeastOnceProcessor proc = processor.getAndSet(null); + final AtLeastOnceProcessorInstrumented proc = processor.getAndSet(null); if (proc != null) { proc.stopAsync(); } @@ -123,4 +123,8 @@ protected void triggerShutdown() { public void pause() { latch.reset(); } + + public void preventFlush() { + processor.get().preventFlush(); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 9e846ccf..452a735c 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -52,6 +52,12 @@ public void pause(int... instanceIds) { }); } + public void preventFlush(int... instanceIds) { + IntStream.of(instanceIds).parallel().forEach(instanceId -> { + workers.get(instanceId).preventFlush(); + }); + } + /** * Processors are guaranteed to not process events after this method returns. */ From bb00ed4e0609d4fb796fa98d0788abf8861daa13 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 18:56:47 +0000 Subject: [PATCH 45/78] Add pause/unpause to AtLeastOnceProcessorInstrumented. Add failing tests handleExceptionDuringFlushTest and killAndRestart1of1ForcingDuplicatesTest. Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessorInstrumented.java | 15 ++++-- .../StreamProcessingTest.java | 51 ++++++++++++++----- .../streamprocessing/WorkerProcess.java | 8 ++- 3 files changed, 53 insertions(+), 21 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 0621ddf3..fe27d5be 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -16,7 +16,8 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writer; - private final ReusableLatch latch; + + private final ReusableLatch latch = new ReusableLatch(true); private final AtomicLong unflushedEventCount = new AtomicLong(0); private final AtomicBoolean preventFlushFlag = new AtomicBoolean(false); @@ -25,12 +26,10 @@ public AtLeastOnceProcessorInstrumented( Supplier> reader, long readTimeoutMillis, int instanceId, - EventStreamWriter writer, - ReusableLatch latch) { + EventStreamWriter writer) { super(pruner, reader, readTimeoutMillis); this.instanceId = instanceId; this.writer = writer; - this.latch = latch; } @Override @@ -65,6 +64,14 @@ protected void injectFault(ReaderGroupPruner pruner) throws Exception { } } + public void pause() { + latch.reset(); + } + + public void unpause() { + latch.release(); + } + public void preventFlush() { preventFlushFlag.set(true); } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 1b4bd97e..6ec22a4d 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -62,6 +62,14 @@ public static void tearDown() throws Exception { SETUP_UTILS.get().stopAllServices(); } + void sleep(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + @RequiredArgsConstructor static class TestContext { final EventStreamWriter writer; @@ -281,11 +289,7 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. - try { - Thread.sleep(2*ctx.checkpointPeriodMs); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + sleep(2*ctx.checkpointPeriodMs); // Kill the worker instance. ctx.workerProcessGroup.pause(0); // Start a new worker instance. It should identify the dead worker and call readerOffline(null). @@ -296,6 +300,33 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { }); } + @Test + public void handleExceptionDuringFlushTest() throws Exception { + endToEndTest(6, 24, 1, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + sleep(2*ctx.checkpointPeriodMs); + // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. + // If this happens, we will throw an exception and this test will fail. This should be rare. + ctx.workerProcessGroup.preventFlush(0); + // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. + final int expectedDuplicateEventCount = 3; + writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while so that flush is called and throws an exception. + // This will close the reader (with what value for readerOffline?). + sleep(2*ctx.checkpointPeriodMs); + // Start a new worker instance so that we can determine where it reads from. + // The new worker should produce duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 100, new int[]{1}); + Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + }); + } + @Test public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { endToEndTest(6, 24, 1, ctx -> { @@ -304,11 +335,7 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. - try { - Thread.sleep(2*ctx.checkpointPeriodMs); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + sleep(2*ctx.checkpointPeriodMs); // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. // If this happens, we will throw an exception and this test will fail. This should be rare. ctx.workerProcessGroup.preventFlush(0); @@ -316,17 +343,17 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { final int expectedDuplicateEventCount = 3; writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + sleep(1*ctx.checkpointPeriodMs); // Kill the worker instance. ctx.workerProcessGroup.pause(0); // Start a new worker instance. It should identify the dead worker and call readerOffline(null). // The new worker should produce duplicates. ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 3, new int[]{1}); + writeEventsAndValidate(ctx, 100, new int[]{1}); Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); }); } - @Test public void kill5of6Test() throws Exception { endToEndTest(6, 24, 6, ctx -> { diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index f565a46d..09302c76 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -42,7 +42,6 @@ public class WorkerProcess extends AbstractExecutionThreadService { private final WorkerProcessConfig config; private final int instanceId; - private final ReusableLatch latch = new ReusableLatch(true); private final AtomicReference processor = new AtomicReference<>(); @@ -100,8 +99,7 @@ protected void run() throws Exception { ReaderConfig.builder().build()), config.readTimeoutMillis, instanceId, - writer, - latch); + writer); processor.set(proc); proc.startAsync(); proc.awaitTerminated(); @@ -115,13 +113,13 @@ protected void triggerShutdown() { final AtLeastOnceProcessorInstrumented proc = processor.getAndSet(null); if (proc != null) { proc.stopAsync(); + proc.unpause(); } - latch.release(); log.info("triggerShutdown: END"); } public void pause() { - latch.reset(); + processor.get().pause(); } public void preventFlush() { From 52e5caa3487998e9f2998f0e5c78b16b803cd324 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 22:16:55 +0000 Subject: [PATCH 46/78] Add different write modes. Split writeEventsAndValidate. Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 4 +- .../AtLeastOnceProcessorInstrumented.java | 33 +++++++- .../StreamProcessingTest.java | 75 +++++++++++++++---- .../streamprocessing/WorkerProcess.java | 6 +- .../streamprocessing/WorkerProcessConfig.java | 1 + .../streamprocessing/WorkerProcessGroup.java | 13 +++- .../example/streamprocessing/WriteMode.java | 17 +++++ 7 files changed, 125 insertions(+), 24 deletions(-) create mode 100644 pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WriteMode.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 5af3447c..997ed354 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -85,13 +85,13 @@ protected void run() throws Exception { * * @param eventRead The event read. */ - abstract public void process(EventRead eventRead); + abstract public void process(EventRead eventRead) throws Exception; /** * If {@link #process} did not completely process prior events, it must do so before returning. * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. */ - public void flush() { + public void flush() throws Exception { } protected void injectFault(ReaderGroupPruner pruner) throws Exception { diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index fe27d5be..2259112d 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -7,8 +7,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor { @@ -20,25 +24,39 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writeModeRef = new AtomicReference<>(); + private final List queue = new ArrayList<>(); public AtLeastOnceProcessorInstrumented( Supplier pruner, Supplier> reader, long readTimeoutMillis, + WriteMode writeMode, int instanceId, EventStreamWriter writer) { super(pruner, reader, readTimeoutMillis); + writeModeRef.set(writeMode); this.instanceId = instanceId; this.writer = writer; } @Override - public void process(EventRead eventRead) { + public void process(EventRead eventRead) throws Exception { final TestEvent event = eventRead.getEvent(); event.processedByInstanceId = instanceId; - log.info("process: event={}", event); - writer.writeEvent(Integer.toString(event.key), event); - unflushedEventCount.incrementAndGet(); + final WriteMode mode = writeModeRef.get(); + log.info("process: mode={}, event={}", mode, event); + if (mode == WriteMode.AlwaysHoldUntilFlushed) { + queue.add(event); + unflushedEventCount.incrementAndGet(); + } else { + final CompletableFuture future = writer.writeEvent(Integer.toString(event.key), event); + if (mode == WriteMode.AlwaysDurable) { + future.get(); + } else { + unflushedEventCount.incrementAndGet(); + } + } } @Override @@ -46,6 +64,9 @@ public void flush() { if (preventFlushFlag.get()) { throw new RuntimeException("Flush called but this test requires that that flush not be called. Try to rerun the test."); } + log.info("flush: Writing {} queued events", queue.size()); + queue.forEach((event) -> writer.writeEvent(Integer.toString(event.key), event)); + queue.clear(); writer.flush(); final long flushedEventCount = unflushedEventCount.getAndSet(0); log.info("flush: Flushed {} events", flushedEventCount); @@ -75,4 +96,8 @@ public void unpause() { public void preventFlush() { preventFlushFlag.set(true); } + + public void setWriteModeRef(WriteMode writeMode) { + writeModeRef.set(writeMode); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 6ec22a4d..7a3e4ce3 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -82,17 +82,24 @@ static class TestContext { /** * Write the given number of events to the Pravega input stream. - * Then read events from the Pravega output stream to ensure that the processors produced the correct result. * * @param ctx provides access to the generator, writer, etc. * @param numEvents number of events to write - * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. */ - void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { - ctx.validator.clearCounters(); - // Write events to input stream. + void writeEvents(TestContext ctx, int numEvents) { Iterators.limit(ctx.generator, numEvents).forEachRemaining(event -> ctx.writer.writeEvent(Integer.toString(event.key), event)); + } + + /** + * Write the given number of events to the Pravega input stream. + * Then read events from the Pravega output stream to ensure that the processors produced the correct result. + * + * @param ctx provides access to the validator, reader, etc. + * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. + */ + void validateEvents(TestContext ctx, int[] expectedInstanceIds) { // Read events from output stream. Return when complete or throw exception if out of order or timeout. + ctx.validator.clearCounters(); ctx.validator.validate(ctx.readerIterator, ctx.generator.getLastSequenceNumbers()); // Confirm that only instances in expectedInstanceIds have processed the events. final Map eventCountByInstanceId = ctx.validator.getEventCountByInstanceId(); @@ -109,6 +116,19 @@ void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstan } } + /** + * Write the given number of events to the Pravega input stream. + * Then read events from the Pravega output stream to ensure that the processors produced the correct result. + * + * @param ctx provides access to the generator, writer, etc. + * @param numEvents number of events to write + * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. + */ + void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstanceIds) { + writeEvents(ctx, numEvents); + validateEvents(ctx, expectedInstanceIds); + } + /** * Write events to a Pravega stream, read events from the same stream, and validate expected ordering. */ @@ -167,7 +187,7 @@ public void noProcessorTest() throws Exception { * @param numInitialInstances number of initial processor instances * @param func function to run to write and write events, etc. */ - private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, Consumer func) throws Exception { + private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, WriteMode writeMode, Consumer func) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); @@ -191,6 +211,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, .membershipSynchronizerStreamName(membershipSynchronizerStreamName) .numSegments(numSegments) .checkpointPeriodMs(checkpointPeriodMs) + .writeMode(writeMode) .build(); @Cleanup final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); @@ -244,7 +265,7 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, @Test public void trivialTest() throws Exception { - endToEndTest(1, 1, 1, ctx -> { + endToEndTest(1, 1, 1, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 20, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }); @@ -252,7 +273,29 @@ public void trivialTest() throws Exception { @Test public void gracefulRestart1of1Test() throws Exception { - endToEndTest(6, 24, 1, ctx -> { + endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }); + } + + @Test + public void gracefulRestart1of1DurableTest() throws Exception { + endToEndTest(6, 24, 1, WriteMode.AlwaysDurable, ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }); + } + + @Test + public void gracefulRestart1of1DHoldUntilFlushedTest() throws Exception { + endToEndTest(6, 24, 1, WriteMode.AlwaysHoldUntilFlushed, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); ctx.workerProcessGroup.stop(0); ctx.workerProcessGroup.start(1); @@ -263,7 +306,7 @@ public void gracefulRestart1of1Test() throws Exception { @Test public void gracefulStop1of2Test() throws Exception { - endToEndTest(6, 24, 2, ctx -> { + endToEndTest(6, 24, 2, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0, 1}); ctx.workerProcessGroup.stop(0); writeEventsAndValidate(ctx, 90, new int[]{1}); @@ -273,7 +316,7 @@ public void gracefulStop1of2Test() throws Exception { @Test public void killAndRestart1of1Test() throws Exception { - endToEndTest(6, 24, 1, ctx -> { + endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); ctx.workerProcessGroup.pause(0); ctx.workerProcessGroup.start(1); @@ -283,7 +326,7 @@ public void killAndRestart1of1Test() throws Exception { @Test public void killAndRestart1of1WhenIdleTest() throws Exception { - endToEndTest(6, 24, 1, ctx -> { + endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. @@ -302,7 +345,7 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { @Test public void handleExceptionDuringFlushTest() throws Exception { - endToEndTest(6, 24, 1, ctx -> { + endToEndTest(6, 24, 1, WriteMode.AlwaysHoldUntilFlushed, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. @@ -314,7 +357,7 @@ public void handleExceptionDuringFlushTest() throws Exception { ctx.workerProcessGroup.preventFlush(0); // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. final int expectedDuplicateEventCount = 3; - writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); + writeEvents(ctx, expectedDuplicateEventCount); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while so that flush is called and throws an exception. // This will close the reader (with what value for readerOffline?). @@ -322,14 +365,14 @@ public void handleExceptionDuringFlushTest() throws Exception { // Start a new worker instance so that we can determine where it reads from. // The new worker should produce duplicates. ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 100, new int[]{1}); + validateEvents(ctx, new int[]{1}); Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); }); } @Test public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { - endToEndTest(6, 24, 1, ctx -> { + endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. @@ -356,7 +399,7 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { @Test public void kill5of6Test() throws Exception { - endToEndTest(6, 24, 6, ctx -> { + endToEndTest(6, 24, 6, WriteMode.Default, ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); ctx.workerProcessGroup.pause(0, 1, 2, 3, 4); writeEventsAndValidate(ctx, 90, new int[]{5}); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index 09302c76..2e114cda 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -25,7 +25,6 @@ import io.pravega.client.stream.Serializer; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; -import io.pravega.common.util.ReusableLatch; import lombok.Builder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -98,6 +97,7 @@ protected void run() throws Exception { serializer, ReaderConfig.builder().build()), config.readTimeoutMillis, + config.writeMode, instanceId, writer); processor.set(proc); @@ -125,4 +125,8 @@ public void pause() { public void preventFlush() { processor.get().preventFlush(); } + + public void setWriteMode(WriteMode mode) { + processor.get().setWriteModeRef(mode); + } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java index 182ab474..1e282385 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java @@ -25,4 +25,5 @@ public class WorkerProcessConfig { @Builder.Default public final long checkpointPeriodMs = 1000; @Builder.Default public final long heartbeatIntervalMillis = 1000; @Builder.Default public final long readTimeoutMillis = 1000; + @Builder.Default public final WriteMode writeMode = WriteMode.Default; } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 452a735c..bcfff73d 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -27,6 +27,7 @@ public class WorkerProcessGroup implements AutoCloseable { private final WorkerProcessConfig config; private final Map workers = new HashMap<>(); + private WriteMode writeMode; /** * Streams are guaranteed to exist after calling this method. @@ -58,6 +59,12 @@ public void preventFlush(int... instanceIds) { }); } + public void setWriteModeAll(WriteMode mode) { + IntStream.of(getInstanceIds()).parallel().forEach(instanceId -> { + workers.get(instanceId).setWriteMode(mode); + }); + } + /** * Processors are guaranteed to not process events after this method returns. */ @@ -75,9 +82,13 @@ public void stop(int... instanceIds) { }); } + protected int[] getInstanceIds() { + return workers.keySet().stream().mapToInt(i -> i).toArray(); + } + public void stopAll() { log.info("stopAll: workers={}", workers); - stop(workers.keySet().stream().mapToInt(i -> i).toArray()); + stop(getInstanceIds()); } @Override diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WriteMode.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WriteMode.java new file mode 100644 index 00000000..0346d7ce --- /dev/null +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WriteMode.java @@ -0,0 +1,17 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +public enum WriteMode { + Default, + AlwaysHoldUntilFlushed, + AlwaysDurable, +} From b2b28734ea9172a9d2203740bfb27b4219cb2dbe Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 9 Jul 2020 22:45:53 +0000 Subject: [PATCH 47/78] Now using EndToEndTestConfig with builder Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessorInstrumented.java | 8 +- .../StreamProcessingTest.java | 287 +++++++++++------- 2 files changed, 177 insertions(+), 118 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 2259112d..2ff72a15 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -44,14 +44,14 @@ public AtLeastOnceProcessorInstrumented( public void process(EventRead eventRead) throws Exception { final TestEvent event = eventRead.getEvent(); event.processedByInstanceId = instanceId; - final WriteMode mode = writeModeRef.get(); - log.info("process: mode={}, event={}", mode, event); - if (mode == WriteMode.AlwaysHoldUntilFlushed) { + final WriteMode writeMode = writeModeRef.get(); + log.info("process: writeMode={}, event={}", writeMode, event); + if (writeMode == WriteMode.AlwaysHoldUntilFlushed) { queue.add(event); unflushedEventCount.incrementAndGet(); } else { final CompletableFuture future = writer.writeEvent(Integer.toString(event.key), event); - if (mode == WriteMode.AlwaysDurable) { + if (writeMode == WriteMode.AlwaysDurable) { future.get(); } else { unflushedEventCount.incrementAndGet(); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 7a3e4ce3..1d2c8339 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -26,6 +26,7 @@ import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; +import lombok.Builder; import lombok.Cleanup; import lombok.RequiredArgsConstructor; import org.junit.AfterClass; @@ -176,18 +177,26 @@ public void noProcessorTest() throws Exception { writeEventsAndValidate(ctx, 15, new int[]{-1}); } + @Builder + protected static class EndToEndTestConfig { + // number of stream segments + @Builder.Default public final int numSegments = 1; + // number of unique routi + @Builder.Default public final int numKeys = 1; + // number of initial processor instances + @Builder.Default public final int numInitialInstances = 1; + @Builder.Default public final WriteMode writeMode = WriteMode.Default; + // function to run to write and write events, etc. + @Builder.Default public final Consumer func = (ctx) -> {}; + } + /** * Write events to the input stream. Start multiple processors which can read from the input stream * and write to the output stream. Validate events in the output stream. * This method performs the setup and teardown. The provided function func performs the actual write and read * of events; stops, pauses, and starts processor instances; and validates results. - * - * @param numSegments number of stream segments - * @param numKeys number of unique routing keys - * @param numInitialInstances number of initial processor instances - * @param func function to run to write and write events, etc. */ - private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, WriteMode writeMode, Consumer func) throws Exception { + private void run(EndToEndTestConfig config) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); @@ -209,15 +218,15 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, .inputStreamName(inputStreamName) .outputStreamName(outputStreamName) .membershipSynchronizerStreamName(membershipSynchronizerStreamName) - .numSegments(numSegments) + .numSegments(config.numSegments) .checkpointPeriodMs(checkpointPeriodMs) - .writeMode(writeMode) + .writeMode(config.writeMode) .build(); @Cleanup final WorkerProcessGroup workerProcessGroup = WorkerProcessGroup.builder().config(workerProcessConfig).build(); // Start initial set of processors. This will also create the necessary streams. - workerProcessGroup.start(IntStream.range(0, numInitialInstances).toArray()); + workerProcessGroup.start(IntStream.range(0, config.numInitialInstances).toArray()); // Prepare generator writer that will write to the stream read by the processor. @Cleanup @@ -245,10 +254,10 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, validationReaderConfig); final long readTimeoutMills = 60000; EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, readTimeoutMills); - final TestEventGenerator generator = new TestEventGenerator(numKeys); + final TestEventGenerator generator = new TestEventGenerator(config.numKeys); final TestEventValidator validator = new TestEventValidator(); final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup, checkpointPeriodMs); - func.accept(ctx); + config.func.accept(ctx); log.info("Cleanup"); workerProcessGroup.close(); @@ -265,144 +274,194 @@ private void endToEndTest(int numSegments, int numKeys, int numInitialInstances, @Test public void trivialTest() throws Exception { - endToEndTest(1, 1, 1, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 20, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(1) + .numKeys(1) + .numInitialInstances(1) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 20, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void gracefulRestart1of1Test() throws Exception { - endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - ctx.workerProcessGroup.stop(0); - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 90, new int[]{1}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void gracefulRestart1of1DurableTest() throws Exception { - endToEndTest(6, 24, 1, WriteMode.AlwaysDurable, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - ctx.workerProcessGroup.stop(0); - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 90, new int[]{1}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysDurable) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void gracefulRestart1of1DHoldUntilFlushedTest() throws Exception { - endToEndTest(6, 24, 1, WriteMode.AlwaysHoldUntilFlushed, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - ctx.workerProcessGroup.stop(0); - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 90, new int[]{1}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysHoldUntilFlushed) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.stop(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void gracefulStop1of2Test() throws Exception { - endToEndTest(6, 24, 2, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0, 1}); - ctx.workerProcessGroup.stop(0); - writeEventsAndValidate(ctx, 90, new int[]{1}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(2) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0, 1}); + ctx.workerProcessGroup.stop(0); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void killAndRestart1of1Test() throws Exception { - endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - ctx.workerProcessGroup.pause(0); - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 90, new int[]{1}); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + ctx.workerProcessGroup.pause(0); + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + }).build()); } @Test public void killAndRestart1of1WhenIdleTest() throws Exception { - endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? - // This will update the reader group state to indicate that this reader has read up to this point. - sleep(2*ctx.checkpointPeriodMs); - // Kill the worker instance. - ctx.workerProcessGroup.pause(0); - // Start a new worker instance. It should identify the dead worker and call readerOffline(null). - // The new worker should resume exactly where the killed worker left off, producing no duplicates. - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 19, new int[]{1}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + sleep(2*ctx.checkpointPeriodMs); + // Kill the worker instance. + ctx.workerProcessGroup.pause(0); + // Start a new worker instance. It should identify the dead worker and call readerOffline(null). + // The new worker should resume exactly where the killed worker left off, producing no duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 19, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void handleExceptionDuringFlushTest() throws Exception { - endToEndTest(6, 24, 1, WriteMode.AlwaysHoldUntilFlushed, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? - // This will update the reader group state to indicate that this reader has read up to this point. - sleep(2*ctx.checkpointPeriodMs); - // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. - // If this happens, we will throw an exception and this test will fail. This should be rare. - ctx.workerProcessGroup.preventFlush(0); - // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. - final int expectedDuplicateEventCount = 3; - writeEvents(ctx, expectedDuplicateEventCount); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Wait for a while so that flush is called and throws an exception. - // This will close the reader (with what value for readerOffline?). - sleep(2*ctx.checkpointPeriodMs); - // Start a new worker instance so that we can determine where it reads from. - // The new worker should produce duplicates. - ctx.workerProcessGroup.start(1); - validateEvents(ctx, new int[]{1}); - Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysHoldUntilFlushed) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + sleep(2*ctx.checkpointPeriodMs); + // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. + // If this happens, we will throw an exception and this test will fail. This should be rare. + ctx.workerProcessGroup.preventFlush(0); + // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. + final int expectedDuplicateEventCount = 3; + writeEvents(ctx, expectedDuplicateEventCount); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while so that flush is called and throws an exception. + // This will close the reader (with what value for readerOffline?). + sleep(2*ctx.checkpointPeriodMs); + // Start a new worker instance so that we can determine where it reads from. + // The new worker should produce duplicates. + ctx.workerProcessGroup.start(1); + validateEvents(ctx, new int[]{1}); + Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { - endToEndTest(6, 24, 1, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? - // This will update the reader group state to indicate that this reader has read up to this point. - sleep(2*ctx.checkpointPeriodMs); - // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. - // If this happens, we will throw an exception and this test will fail. This should be rare. - ctx.workerProcessGroup.preventFlush(0); - // Write some events that will be processed but not flushed. - final int expectedDuplicateEventCount = 3; - writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - sleep(1*ctx.checkpointPeriodMs); - // Kill the worker instance. - ctx.workerProcessGroup.pause(0); - // Start a new worker instance. It should identify the dead worker and call readerOffline(null). - // The new worker should produce duplicates. - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 100, new int[]{1}); - Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysHoldUntilFlushed) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // TODO: Monitor the reader group to determine when this occurs? + // This will update the reader group state to indicate that this reader has read up to this point. + sleep(2*ctx.checkpointPeriodMs); + // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. + // If this happens, we will throw an exception and this test will fail. This should be rare. + ctx.workerProcessGroup.preventFlush(0); + // Write some events that will be processed but not flushed. + final int expectedDuplicateEventCount = 3; + writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + sleep(1*ctx.checkpointPeriodMs); + // Kill the worker instance. + ctx.workerProcessGroup.pause(0); + // Start a new worker instance. It should identify the dead worker and call readerOffline(null). + // The new worker should produce duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 100, new int[]{1}); + Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + }).build()); } @Test public void kill5of6Test() throws Exception { - endToEndTest(6, 24, 6, WriteMode.Default, ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); - ctx.workerProcessGroup.pause(0, 1, 2, 3, 4); - writeEventsAndValidate(ctx, 90, new int[]{5}); - }); + run(EndToEndTestConfig.builder() + .numSegments(6) + .numKeys(24) + .numInitialInstances(6) + .writeMode(WriteMode.Default) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); + ctx.workerProcessGroup.pause(0, 1, 2, 3, 4); + writeEventsAndValidate(ctx, 90, new int[]{5}); + }).build()); } } From 0fc87839e419b3c4c83c1d0bb3edd05e1f55250c Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 10 Jul 2020 00:03:30 +0000 Subject: [PATCH 48/78] Add timeout to failing tests Signed-off-by: Claudio Fahey --- .../example/streamprocessing/StreamProcessingTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 1d2c8339..b7a1ab2f 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -387,7 +387,7 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { }).build()); } - @Test + @Test(timeout = 2*60*1000) public void handleExceptionDuringFlushTest() throws Exception { run(EndToEndTestConfig.builder() .numSegments(6) @@ -419,7 +419,7 @@ public void handleExceptionDuringFlushTest() throws Exception { }).build()); } - @Test + @Test(timeout = 2*60*1000) public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { run(EndToEndTestConfig.builder() .numSegments(6) From a429894e24d9064c8414c8bfe89c40fb31a2b0f6 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 10 Jul 2020 02:12:59 +0000 Subject: [PATCH 49/78] Created failing test handleExceptionDuringProcessTest Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessorInstrumented.java | 8 +++++ .../StreamProcessingTest.java | 32 ++++++++++++++++--- .../streamprocessing/TestEventValidator.java | 6 +++- .../streamprocessing/WorkerProcess.java | 4 +++ .../streamprocessing/WorkerProcessGroup.java | 4 +++ .../utils/EventStreamReaderIterator.java | 15 +++++++-- 6 files changed, 61 insertions(+), 8 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 2ff72a15..26c0ddfb 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -24,6 +24,7 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writeModeRef = new AtomicReference<>(); private final List queue = new ArrayList<>(); @@ -46,6 +47,9 @@ public void process(EventRead eventRead) throws Exception { event.processedByInstanceId = instanceId; final WriteMode writeMode = writeModeRef.get(); log.info("process: writeMode={}, event={}", writeMode, event); + if (induceFailureDuringProcessFlag.get()) { + throw new RuntimeException("induceFailureDuringProcess is set"); + } if (writeMode == WriteMode.AlwaysHoldUntilFlushed) { queue.add(event); unflushedEventCount.incrementAndGet(); @@ -97,6 +101,10 @@ public void preventFlush() { preventFlushFlag.set(true); } + public void induceFailureDuringProcess() { + induceFailureDuringProcessFlag.set(true); + } + public void setWriteModeRef(WriteMode writeMode) { writeModeRef.set(writeMode); } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index b7a1ab2f..73c93359 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -99,6 +99,7 @@ void writeEvents(TestContext ctx, int numEvents) { * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. */ void validateEvents(TestContext ctx, int[] expectedInstanceIds) { + log.info("validateEvents: BEGIN"); // Read events from output stream. Return when complete or throw exception if out of order or timeout. ctx.validator.clearCounters(); ctx.validator.validate(ctx.readerIterator, ctx.generator.getLastSequenceNumbers()); @@ -106,15 +107,16 @@ void validateEvents(TestContext ctx, int[] expectedInstanceIds) { final Map eventCountByInstanceId = ctx.validator.getEventCountByInstanceId(); final Set actualInstanceIds = eventCountByInstanceId.keySet(); final Set expectedInstanceIdsSet = Arrays.stream(expectedInstanceIds).boxed().collect(Collectors.toCollection(HashSet::new)); - log.info("writeEventsAndValidate: eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); + log.info("validateEvents: eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); Assert.assertTrue(MessageFormat.format("eventCountByInstanceId={0}, expectedInstanceIdsSet={1}", eventCountByInstanceId, expectedInstanceIdsSet), Sets.difference(actualInstanceIds, expectedInstanceIdsSet).isEmpty()); // Warn if any instances are idle. This cannot be an assertion because this may happen under normal conditions. final Sets.SetView idleInstanceIds = Sets.difference(expectedInstanceIdsSet, actualInstanceIds); if (!idleInstanceIds.isEmpty()) { - log.warn("writeEventsAndValidate: Some instances processed no events; eventCountByInstanceId={}, expectedInstanceIdsSet={}", + log.warn("validateEvents: Some instances processed no events; eventCountByInstanceId={}, expectedInstanceIdsSet={}", eventCountByInstanceId, expectedInstanceIdsSet); } + log.info("validateEvents: END"); } /** @@ -387,6 +389,28 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { }).build()); } + @Test(timeout = 2*60*1000) + public void handleExceptionDuringProcessTest() throws Exception { + run(EndToEndTestConfig.builder() + .numSegments(1) + .numKeys(1) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysHoldUntilFlushed) + .func(ctx -> { + // Force process function to throw an exception. + ctx.workerProcessGroup.get(0).induceFailureDuringProcess(); + // Write an event. + writeEvents(ctx, 1); + // Wait for process function to throw an exception. + sleep(2*ctx.checkpointPeriodMs); + // Start a new worker instance so that we can determine where it reads from. + // The event should have been processed exactly once. + ctx.workerProcessGroup.start(1); + validateEvents(ctx, new int[]{1}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + }).build()); + } + @Test(timeout = 2*60*1000) public void handleExceptionDuringFlushTest() throws Exception { run(EndToEndTestConfig.builder() @@ -401,13 +425,11 @@ public void handleExceptionDuringFlushTest() throws Exception { // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. sleep(2*ctx.checkpointPeriodMs); - // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. - // If this happens, we will throw an exception and this test will fail. This should be rare. + // Force an exception during flush. ctx.workerProcessGroup.preventFlush(0); // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. final int expectedDuplicateEventCount = 3; writeEvents(ctx, expectedDuplicateEventCount); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while so that flush is called and throws an exception. // This will close the reader (with what value for readerOffline?). sleep(2*ctx.checkpointPeriodMs); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index 821e2369..ae2950e5 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -13,6 +13,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.text.MessageFormat; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -61,7 +62,10 @@ public void validate(Iterator events, Map expectedLast } } } - throw new NoMoreEventsException("No more events but all expected events were not received"); + throw new NoMoreEventsException(MessageFormat.format( + "No more events but all expected events were not received; " + + "receivedSequenceNumbers={0}, expectedLastSequenceNumbers={1}", + receivedSequenceNumbers, expectedLastSequenceNumbers)); } public void clearCounters() { diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index 2e114cda..c20a2a3f 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -126,6 +126,10 @@ public void preventFlush() { processor.get().preventFlush(); } + public void induceFailureDuringProcess() { + processor.get().induceFailureDuringProcess(); + } + public void setWriteMode(WriteMode mode) { processor.get().setWriteModeRef(mode); } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index bcfff73d..16f7a83d 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -29,6 +29,10 @@ public class WorkerProcessGroup implements AutoCloseable { private final Map workers = new HashMap<>(); private WriteMode writeMode; + public WorkerProcess get(int instanceId) { + return workers.get(instanceId); + } + /** * Streams are guaranteed to exist after calling this method. */ diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java index f974c102..8104a1e1 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -12,6 +12,8 @@ import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.Optional; @@ -20,6 +22,8 @@ * An Iterator for reading from a Pravega stream. */ public class EventStreamReaderIterator implements Iterator { + static final Logger log = LoggerFactory.getLogger(EventStreamReaderIterator.class); + private final EventStreamReader reader; private final long timeoutMillis; private Optional nextEvent = Optional.empty(); @@ -48,16 +52,23 @@ public T next() { } private void readIfNeeded() { + log.info("readIfNeeded: BEGIN"); if (!nextEvent.isPresent()) { - for (; ; ) { - final EventRead eventRead = reader.readNextEvent(timeoutMillis); + final long t0 = System.nanoTime(); + long nextTimeoutMillis = timeoutMillis; + while (nextTimeoutMillis >= 0) { + log.info("readIfNeeded: nextTimeoutMillis={}", nextTimeoutMillis); + final EventRead eventRead = reader.readNextEvent(nextTimeoutMillis); + log.info("readIfNeeded: eventRead={}", eventRead); if (!eventRead.isCheckpoint()) { if (eventRead.getEvent() != null) { nextEvent = Optional.of(eventRead.getEvent()); } return; } + nextTimeoutMillis = timeoutMillis - (System.nanoTime() - t0) / 1000 / 1000; } } + log.info("readIfNeeded: END"); } } From e34bb9088c2c66d6fe787a4484490093751a222e Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 10 Jul 2020 05:56:08 +0000 Subject: [PATCH 50/78] Bug fix: Fix handling of position when an exception occurs during processing or flushing Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 56 +++++++++++-------- .../example/streamprocessing/README.md | 17 ++++-- 2 files changed, 45 insertions(+), 28 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index 997ed354..b98e186c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -43,10 +43,6 @@ public AtLeastOnceProcessor(Supplier prunerSupplier, Supplier /** * Run the event processor loop. - * - * If the previous call to readNextEvent returned a checkpoint, the next call - * to readNextEvent will record in the reader group that this reader - * has read and processed events up to the previous {@link Position}. */ @Override protected void run() throws Exception { @@ -54,26 +50,42 @@ protected void run() throws Exception { // before the EventStreamReader is created. Otherwise, another ReaderGroupPruner instance may place this reader offline. // It is also critical that when this method stops running the ReaderGroupPruner is eventually stopped so that // it no longer sends heartbeats. - try (final ReaderGroupPruner pruner = prunerSupplier.get(); - final EventStreamReader reader = readerSupplier.get()) { - while (isRunning()) { - final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); - log.info("eventRead={}", eventRead); - // We must inject the fault between read and process. - // This ensures that a *new* event cannot be processed after the fault injection latch is set. - injectFault(pruner); - if (eventRead.isCheckpoint()) { - flush(); - } else if (eventRead.getEvent() != null) { - process(eventRead); + try (final ReaderGroupPruner pruner = prunerSupplier.get()) { + final EventStreamReader reader = readerSupplier.get(); + Position lastProcessedPosition = null; + Position lastFlushedPosition = null; + try { + while (isRunning()) { + final EventRead eventRead = reader.readNextEvent(readTimeoutMillis); + log.info("eventRead={}", eventRead); + // We must inject the fault between read and process. + // This ensures that a *new* event cannot be processed after the fault injection latch is set. + injectFault(pruner); + if (eventRead.isCheckpoint()) { + flush(); + lastProcessedPosition = eventRead.getPosition(); + lastFlushedPosition = lastProcessedPosition; + } else if (eventRead.getEvent() != null) { + try { + process(eventRead); + lastProcessedPosition = eventRead.getPosition(); + } catch (Exception e) { + // If an exception occurs during processing, attempt to flush. + flush(); + lastFlushedPosition = lastProcessedPosition; + throw e; + } + } } + // Gracefully stop. + log.info("Stopping"); + flush(); + lastFlushedPosition = lastProcessedPosition; + } finally { + log.info("Closing reader"); + // Note that if lastFlushedPosition is null, the segment offset used by a future reader will remain unchanged. + reader.closeAt(lastFlushedPosition); } - // Gracefully stop. - // Call readNextEvent to indicate that the previous event was processed. - // When the reader is closed, it will call readerOffline with the proper position. - log.info("Stopping"); - reader.readNextEvent(0); - flush(); } log.info("Stopped"); } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index ed7bc55c..a4f68b22 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -112,16 +112,21 @@ The interval can be specified using `ReaderGroupConfig.automaticCheckpointInterv ## Graceful shutdown A graceful shutdown of an at-least-once process requires that it call -`ReaderGroup.readerOffline(readerId, lastPosition)`, where `lastPosition` +`EventStreamReader.closeAt(position)`, where `position` is the position object of the last event that was successfully processed. -This method is called automatically when the `EventStreamReader` is closed. +If `position` is null, this will indicate that the reader did not successfully process any events. When graceful shutdowns occur, events will be processed exactly once. +## Exceptional shutdown + +An exceptional shutdown occurs when `AtLeastOnceProcessor.run()` encounters a fatal exception. +This is handled similarly to the graceful shutdown case. + ## Ungraceful shutdown An ungraceful shutdown of a process can occur when the host abruptly loses power, when it loses its network connection, or when the process is terminated with `kill -9`. -In these situations, the process is unable to successfully call `ReaderGroup.readerOffline()`. +In these situations, the process is unable to successfully call `EventStreamReader.closeAt(position)`. However, the Reader Group guarantees that any assigned segments will remain assigned to the reader until `readerOffline()` is called. If no action is taken, the events in the segments assigned to the dead worker's reader will never be read. @@ -139,9 +144,9 @@ If writes are not idempotent, this will produce duplicates. In order to detect dead workers, each worker process must run an instance of `ReaderGroupPruner`. `ReaderGroupPruner` uses a [MembershipSynchronizer](MembershipSynchronizer.java) which uses a [State Synchronizer](http://pravega.io/docs/latest/state-synchronizer-design/) to -maintain the set of workers that are providing heart beats. -Each worker sends a heart beat by adding an update to the `MembershipSynchronizer`'s State Synchronizer. -Workers that fail to provide a heart beat after 10 intervals will be removed from the `MembershipSynchronizer`. +maintain the set of workers that are providing heartbeats. +Each worker sends a heartbeat by adding an update to the `MembershipSynchronizer`'s State Synchronizer. +Workers that fail to provide a heartbeat after 10 intervals will be removed from the `MembershipSynchronizer`. Finally, any workers in the Reader Group that are not in the `MembershipSynchronizer` are considered dead and `readerOffline(readerId, null)` will be called by one or more instances of `ReaderGroupPruner`. From 265fd13a604f6a4d5f65e22d2bad8e80d0a5f93f Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 10 Jul 2020 05:58:32 +0000 Subject: [PATCH 51/78] Fix killAndRestart1of1ForcingDuplicatesTest Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 2 + .../AtLeastOnceProcessorInstrumented.java | 27 ++++++------ .../StreamProcessingTest.java | 42 +++++++++---------- .../streamprocessing/WorkerProcess.java | 6 +-- .../streamprocessing/WorkerProcessGroup.java | 30 ++++++------- .../utils/EventStreamReaderIterator.java | 4 -- 6 files changed, 53 insertions(+), 58 deletions(-) diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 2d8e010a..96cf4db4 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -22,10 +22,12 @@ resources ext { junitVersion = '4.11' commonsCLIVersion = '1.4' + junitParamsVersion = '1.1.1' } dependencies { testCompile "junit:junit:${junitVersion}" + testCompile "pl.pragmatists:JUnitParams:${junitParamsVersion}" testCompile "io.pravega:pravega-standalone:${pravegaVersion}" testCompile "io.pravega:pravega-test-testcommon:${pravegaVersion}" compileOnly "org.projectlombok:lombok:${lombokVersion}" diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 26c0ddfb..91551dbf 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -21,9 +21,9 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writer; - private final ReusableLatch latch = new ReusableLatch(true); + private final ReusableLatch pauseLatch = new ReusableLatch(true); private final AtomicLong unflushedEventCount = new AtomicLong(0); - private final AtomicBoolean preventFlushFlag = new AtomicBoolean(false); + private final AtomicBoolean induceFailureDuringFlushFlag = new AtomicBoolean(false); private final AtomicBoolean induceFailureDuringProcessFlag = new AtomicBoolean(false); private final AtomicReference writeModeRef = new AtomicReference<>(); private final List queue = new ArrayList<>(); @@ -65,8 +65,9 @@ public void process(EventRead eventRead) throws Exception { @Override public void flush() { - if (preventFlushFlag.get()) { - throw new RuntimeException("Flush called but this test requires that that flush not be called. Try to rerun the test."); + if (induceFailureDuringFlushFlag.get()) { + log.warn("induceFailureDuringFlushFlag is set"); + throw new RuntimeException("induceFailureDuringFlushFlag is set"); } log.info("flush: Writing {} queued events", queue.size()); queue.forEach((event) -> writer.writeEvent(Integer.toString(event.key), event)); @@ -78,34 +79,34 @@ public void flush() { @Override protected void injectFault(ReaderGroupPruner pruner) throws Exception { - if (!latch.isReleased()) { + if (!pauseLatch.isReleased()) { log.warn("injectFault: BEGIN"); // Pause pruner (but do not close it). This will also pause the membership synchronizer. pruner.pause(); // Halt this processor thread until the latch is released. - latch.await(); + pauseLatch.await(); pruner.unpause(); log.warn("injectFault: END"); } } public void pause() { - latch.reset(); + pauseLatch.reset(); } public void unpause() { - latch.release(); - } - - public void preventFlush() { - preventFlushFlag.set(true); + pauseLatch.release(); } public void induceFailureDuringProcess() { induceFailureDuringProcessFlag.set(true); } - public void setWriteModeRef(WriteMode writeMode) { + public void induceFailureDuringFlush() { + induceFailureDuringFlushFlag.set(true); + } + + public void setWriteMode(WriteMode writeMode) { writeModeRef.set(writeMode); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 73c93359..38c5525e 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -26,6 +26,8 @@ import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; import lombok.Builder; import lombok.Cleanup; import lombok.RequiredArgsConstructor; @@ -33,6 +35,7 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +50,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +@RunWith(JUnitParamsRunner.class) public class StreamProcessingTest { static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); @@ -359,7 +363,7 @@ public void killAndRestart1of1Test() throws Exception { .writeMode(WriteMode.Default) .func(ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); - ctx.workerProcessGroup.pause(0); + ctx.workerProcessGroup.get(0).pause(); ctx.workerProcessGroup.start(1); writeEventsAndValidate(ctx, 90, new int[]{1}); }).build()); @@ -376,11 +380,10 @@ public void killAndRestart1of1WhenIdleTest() throws Exception { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. sleep(2*ctx.checkpointPeriodMs); // Kill the worker instance. - ctx.workerProcessGroup.pause(0); + ctx.workerProcessGroup.get(0).pause(); // Start a new worker instance. It should identify the dead worker and call readerOffline(null). // The new worker should resume exactly where the killed worker left off, producing no duplicates. ctx.workerProcessGroup.start(1); @@ -422,53 +425,46 @@ public void handleExceptionDuringFlushTest() throws Exception { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. sleep(2*ctx.checkpointPeriodMs); // Force an exception during flush. - ctx.workerProcessGroup.preventFlush(0); - // Write some events that will be processed, written to Pravega, and read by the validator, but not explicitly flushed. - final int expectedDuplicateEventCount = 3; - writeEvents(ctx, expectedDuplicateEventCount); + ctx.workerProcessGroup.get(0).induceFailureDuringFlush(); + // Write some events that will be processed and queued for writing (they will not be written though). + final int newEventCount = 3; + writeEvents(ctx, newEventCount); // Wait for a while so that flush is called and throws an exception. - // This will close the reader (with what value for readerOffline?). sleep(2*ctx.checkpointPeriodMs); // Start a new worker instance so that we can determine where it reads from. - // The new worker should produce duplicates. ctx.workerProcessGroup.start(1); validateEvents(ctx, new int[]{1}); - Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }).build()); } - @Test(timeout = 2*60*1000) public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { run(EndToEndTestConfig.builder() - .numSegments(6) - .numKeys(24) + .numSegments(1) + .numKeys(1) .numInitialInstances(1) - .writeMode(WriteMode.AlwaysHoldUntilFlushed) + .writeMode(WriteMode.AlwaysDurable) .func(ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // TODO: Monitor the reader group to determine when this occurs? // This will update the reader group state to indicate that this reader has read up to this point. sleep(2*ctx.checkpointPeriodMs); - // Although we don't have control over when a checkpoint request is received by a reader, we can detect it. - // If this happens, we will throw an exception and this test will fail. This should be rare. - ctx.workerProcessGroup.preventFlush(0); - // Write some events that will be processed but not flushed. + // Write some events that will be processed and durably written. final int expectedDuplicateEventCount = 3; writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); sleep(1*ctx.checkpointPeriodMs); // Kill the worker instance. - ctx.workerProcessGroup.pause(0); + // TODO: We hope this will happen before flushing. + ctx.workerProcessGroup.get(0).pause(); // Start a new worker instance. It should identify the dead worker and call readerOffline(null). // The new worker should produce duplicates. ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 100, new int[]{1}); + writeEventsAndValidate(ctx, 90, new int[]{1}); Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); }).build()); } @@ -482,7 +478,7 @@ public void kill5of6Test() throws Exception { .writeMode(WriteMode.Default) .func(ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); - ctx.workerProcessGroup.pause(0, 1, 2, 3, 4); + IntStream.rangeClosed(0, 4).forEach(i -> ctx.workerProcessGroup.get(i).pause()); writeEventsAndValidate(ctx, 90, new int[]{5}); }).build()); } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java index c20a2a3f..78c4fec9 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcess.java @@ -122,8 +122,8 @@ public void pause() { processor.get().pause(); } - public void preventFlush() { - processor.get().preventFlush(); + public void induceFailureDuringFlush() { + processor.get().induceFailureDuringFlush(); } public void induceFailureDuringProcess() { @@ -131,6 +131,6 @@ public void induceFailureDuringProcess() { } public void setWriteMode(WriteMode mode) { - processor.get().setWriteModeRef(mode); + processor.get().setWriteMode(mode); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index 16f7a83d..e859669f 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -51,23 +51,23 @@ public void start(int... instanceIds) { /** * Processors are guaranteed to not process events written after this method returns. */ - public void pause(int... instanceIds) { - IntStream.of(instanceIds).parallel().forEach(instanceId -> { - workers.get(instanceId).pause(); - }); - } +// public void pause(int... instanceIds) { +// IntStream.of(instanceIds).parallel().forEach(instanceId -> { +// workers.get(instanceId).pause(); +// }); +// } - public void preventFlush(int... instanceIds) { - IntStream.of(instanceIds).parallel().forEach(instanceId -> { - workers.get(instanceId).preventFlush(); - }); - } +// public void preventFlush(int... instanceIds) { +// IntStream.of(instanceIds).parallel().forEach(instanceId -> { +// workers.get(instanceId).induceFailureDuringFlush(); +// }); +// } - public void setWriteModeAll(WriteMode mode) { - IntStream.of(getInstanceIds()).parallel().forEach(instanceId -> { - workers.get(instanceId).setWriteMode(mode); - }); - } +// public void setWriteModeAll(WriteMode mode) { +// IntStream.of(getInstanceIds()).parallel().forEach(instanceId -> { +// workers.get(instanceId).setWriteMode(mode); +// }); +// } /** * Processors are guaranteed to not process events after this method returns. diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java index 8104a1e1..0b28be21 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/EventStreamReaderIterator.java @@ -52,14 +52,11 @@ public T next() { } private void readIfNeeded() { - log.info("readIfNeeded: BEGIN"); if (!nextEvent.isPresent()) { final long t0 = System.nanoTime(); long nextTimeoutMillis = timeoutMillis; while (nextTimeoutMillis >= 0) { - log.info("readIfNeeded: nextTimeoutMillis={}", nextTimeoutMillis); final EventRead eventRead = reader.readNextEvent(nextTimeoutMillis); - log.info("readIfNeeded: eventRead={}", eventRead); if (!eventRead.isCheckpoint()) { if (eventRead.getEvent() != null) { nextEvent = Optional.of(eventRead.getEvent()); @@ -69,6 +66,5 @@ private void readIfNeeded() { nextTimeoutMillis = timeoutMillis - (System.nanoTime() - t0) / 1000 / 1000; } } - log.info("readIfNeeded: END"); } } From 6aab6bb7b558f2a43cfa36ff8ee793ca994377b4 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Tue, 4 Aug 2020 04:04:48 +0000 Subject: [PATCH 52/78] Updated documentation. Now using in-process Pravega cluster. Removed unnecessary sleep from killAndRestart1of1ForcingDuplicatesTest. Signed-off-by: Claudio Fahey --- .../AtLeastOnceProcessor.java | 2 + .../MembershipSynchronizer.java | 6 +++ .../streamprocessing/ReaderGroupPruner.java | 6 +++ .../AtLeastOnceProcessorInstrumented.java | 4 ++ .../StreamProcessingTest.java | 54 +++++++++++++++---- 5 files changed, 63 insertions(+), 9 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java index b98e186c..dcb23a57 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceProcessor.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.AbstractExecutionThreadService; import io.pravega.client.stream.EventRead; import io.pravega.client.stream.EventStreamReader; @@ -106,6 +107,7 @@ protected void run() throws Exception { public void flush() throws Exception { } + @VisibleForTesting protected void injectFault(ReaderGroupPruner pruner) throws Exception { } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java index 227d6bcf..8e6ac71c 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/MembershipSynchronizer.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.AbstractService; import io.pravega.client.SynchronizerClientFactory; @@ -279,11 +280,16 @@ protected void doStop() { task.cancel(false); } + @VisibleForTesting public void pause() { log.warn("paused"); task.cancel(false); } + /** + * Not implemented. + */ + @VisibleForTesting public void unpause() { log.warn("unpause"); } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index f5c67f2f..54e59c1f 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -10,6 +10,7 @@ */ package io.pravega.example.streamprocessing; +import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.AbstractService; import io.pravega.client.SynchronizerClientFactory; import io.pravega.client.stream.ReaderGroup; @@ -93,12 +94,17 @@ protected void doStop() { membershipSynchronizer.stopAsync(); } + @VisibleForTesting public void pause() { log.warn("paused"); task.cancel(false); membershipSynchronizer.pause(); } + /** + * Not implemented. + */ + @VisibleForTesting public void unpause() { log.warn("unpause"); membershipSynchronizer.unpause(); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java index 91551dbf..ee8e0425 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/AtLeastOnceProcessorInstrumented.java @@ -15,6 +15,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +/** + * An AtLeastOnceProcessor that is instrumented for testing. + */ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceProcessorInstrumented.class); @@ -26,6 +29,7 @@ public class AtLeastOnceProcessorInstrumented extends AtLeastOnceProcessor writeModeRef = new AtomicReference<>(); + // A queue containing unflushed events. private final List queue = new ArrayList<>(); public AtLeastOnceProcessorInstrumented( diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 38c5525e..210bc831 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -27,7 +27,6 @@ import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; import junitparams.JUnitParamsRunner; -import junitparams.Parameters; import lombok.Builder; import lombok.Cleanup; import lombok.RequiredArgsConstructor; @@ -50,7 +49,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -@RunWith(JUnitParamsRunner.class) public class StreamProcessingTest { static final Logger log = LoggerFactory.getLogger(StreamProcessingTest.class); @@ -58,7 +56,7 @@ public class StreamProcessingTest { @BeforeClass public static void setup() throws Exception { - SETUP_UTILS.set(new SetupUtils("tcp://localhost:9090")); + SETUP_UTILS.set(new SetupUtils()); SETUP_UTILS.get().startAllServices(); } @@ -96,8 +94,7 @@ void writeEvents(TestContext ctx, int numEvents) { } /** - * Write the given number of events to the Pravega input stream. - * Then read events from the Pravega output stream to ensure that the processors produced the correct result. + * Read events from the Pravega output stream to ensure that the processors produced the correct result. * * @param ctx provides access to the validator, reader, etc. * @param expectedInstanceIds All read events must have a processedByInstanceId in this set. @@ -192,7 +189,7 @@ protected static class EndToEndTestConfig { // number of initial processor instances @Builder.Default public final int numInitialInstances = 1; @Builder.Default public final WriteMode writeMode = WriteMode.Default; - // function to run to write and write events, etc. + // test-specific function to write and validate events, etc. @Builder.Default public final Consumer func = (ctx) -> {}; } @@ -278,6 +275,10 @@ private void run(EndToEndTestConfig config) throws Exception { streamManager.deleteStream(scope, membershipSynchronizerStreamName); } + /** + * Write 20 events with 1 routing key, run 1 processor instance, and validate results. + * @throws Exception + */ @Test public void trivialTest() throws Exception { run(EndToEndTestConfig.builder() @@ -291,6 +292,9 @@ public void trivialTest() throws Exception { }).build()); } + /** + * Gracefully stop 1 of 1 processor instances. + */ @Test public void gracefulRestart1of1Test() throws Exception { run(EndToEndTestConfig.builder() @@ -307,6 +311,10 @@ public void gracefulRestart1of1Test() throws Exception { }).build()); } + /** + * Gracefully stop 1 of 1 processor instances. + * Force each events to be durably written immediately. + */ @Test public void gracefulRestart1of1DurableTest() throws Exception { run(EndToEndTestConfig.builder() @@ -323,6 +331,10 @@ public void gracefulRestart1of1DurableTest() throws Exception { }).build()); } + /** + * Gracefully stop 1 of 1 processor instances. + * Do not writes events to Pravega until flushed. + */ @Test public void gracefulRestart1of1DHoldUntilFlushedTest() throws Exception { run(EndToEndTestConfig.builder() @@ -339,6 +351,9 @@ public void gracefulRestart1of1DHoldUntilFlushedTest() throws Exception { }).build()); } + /** + * Gracefully stop 1 of 2 processor instances. + */ @Test public void gracefulStop1of2Test() throws Exception { run(EndToEndTestConfig.builder() @@ -354,6 +369,11 @@ public void gracefulStop1of2Test() throws Exception { }).build()); } + /** + * Kill and restart 1 of 1 processor instances. + * This is simulated by pausing the first instance until the test completes. + * This may produce duplicates. + */ @Test public void killAndRestart1of1Test() throws Exception { run(EndToEndTestConfig.builder() @@ -366,9 +386,17 @@ public void killAndRestart1of1Test() throws Exception { ctx.workerProcessGroup.get(0).pause(); ctx.workerProcessGroup.start(1); writeEventsAndValidate(ctx, 90, new int[]{1}); + log.info("getDuplicateEventCount={}", ctx.validator.getDuplicateEventCount()); }).build()); } + /** + * Kill and restart 1 of 1 processor instances. + * This is simulated by pausing the first instance until the test completes. + * This tests waits for 2x the checkpoint interval to ensure that the + * first processor writes its position before pausing. + * This will not produce duplicates. + */ @Test public void killAndRestart1of1WhenIdleTest() throws Exception { run(EndToEndTestConfig.builder() @@ -440,6 +468,11 @@ public void handleExceptionDuringFlushTest() throws Exception { Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); }).build()); } + + /** + * This test processes events and then kills the processor before it receives a checkpoint. + * This is expected to produce duplicates. + */ @Test(timeout = 2*60*1000) public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { run(EndToEndTestConfig.builder() @@ -457,9 +490,9 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { final int expectedDuplicateEventCount = 3; writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - sleep(1*ctx.checkpointPeriodMs); - // Kill the worker instance. - // TODO: We hope this will happen before flushing. + // Kill the worker instance before it checkpoints and updates the reader group state. + // There is no control mechanism to prevent the checkpoint. + // If a checkpoint occurs here, this test will fail because duplicates will not be produced. ctx.workerProcessGroup.get(0).pause(); // Start a new worker instance. It should identify the dead worker and call readerOffline(null). // The new worker should produce duplicates. @@ -469,6 +502,9 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { }).build()); } + /** + * Kill 5 of 6 processor instances. + */ @Test public void kill5of6Test() throws Exception { run(EndToEndTestConfig.builder() From 0439f09b3b5a22591cf45e44835205f8ef6247e3 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Tue, 4 Aug 2020 04:13:13 +0000 Subject: [PATCH 53/78] Update README. Signed-off-by: Claudio Fahey --- .../src/main/java/io/pravega/example/streamprocessing/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index a4f68b22..c635c9c9 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -137,7 +137,7 @@ Each worker process runs an instance of `ReaderGroupPruner`. When it detects a dead worker, it calls `readerOffline(readerId, lastPosition)` with a null value for `lastPosition`. The null value for `lastPosition` indicates that it should use the position stored in the -last `ReaderGroupState.CheckpointReader` event in the Reader Group. +last `ReaderGroupState.CheckpointReader` event written by that reader in the Reader Group. Any events processed by the now-dead worker after the last checkpoint will be reprocessed by other workers. If writes are not idempotent, this will produce duplicates. From 1b5d275f5ac083b857af37f04939a063578798bc Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Tue, 4 Aug 2020 04:38:35 +0000 Subject: [PATCH 54/78] Remove unneeded dependency JUnitParams. Signed-off-by: Claudio Fahey --- pravega-client-examples/build.gradle | 2 -- .../pravega/example/streamprocessing/StreamProcessingTest.java | 2 -- 2 files changed, 4 deletions(-) diff --git a/pravega-client-examples/build.gradle b/pravega-client-examples/build.gradle index 96cf4db4..2d8e010a 100644 --- a/pravega-client-examples/build.gradle +++ b/pravega-client-examples/build.gradle @@ -22,12 +22,10 @@ resources ext { junitVersion = '4.11' commonsCLIVersion = '1.4' - junitParamsVersion = '1.1.1' } dependencies { testCompile "junit:junit:${junitVersion}" - testCompile "pl.pragmatists:JUnitParams:${junitParamsVersion}" testCompile "io.pravega:pravega-standalone:${pravegaVersion}" testCompile "io.pravega:pravega-test-testcommon:${pravegaVersion}" compileOnly "org.projectlombok:lombok:${lombokVersion}" diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 210bc831..6f822346 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -26,7 +26,6 @@ import io.pravega.client.stream.Stream; import io.pravega.utils.EventStreamReaderIterator; import io.pravega.utils.SetupUtils; -import junitparams.JUnitParamsRunner; import lombok.Builder; import lombok.Cleanup; import lombok.RequiredArgsConstructor; @@ -34,7 +33,6 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.runner.RunWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From f4a0be83b2d9785f312aaf65c146f94d411dec16 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 21 Aug 2020 04:40:34 +0000 Subject: [PATCH 55/78] Add description to getStream1Name and getStream2Name Signed-off-by: Claudio Fahey --- .../pravega/example/streamprocessing/AppConfiguration.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java index 9b651b03..c131d981 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AppConfiguration.java @@ -34,10 +34,16 @@ public String getInstanceId() { return getEnvVar("INSTANCE_ID", UUID.randomUUID().toString()); } + /** + * The output of EventGenerator and the input of AtLeastOnceApp. + */ public String getStream1Name() { return getEnvVar("PRAVEGA_STREAM_1", "streamprocessing1c"); } + /** + * The output of AtLeastOnceApp and the input of EventDebugSink. + */ public String getStream2Name() { return getEnvVar("PRAVEGA_STREAM_2", "streamprocessing2c"); } From dc9dfcfe5349165d8942530017a11fdf54d55888 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 21 Aug 2020 04:41:24 +0000 Subject: [PATCH 56/78] Randomize execution period of ReaderGroupPruner Signed-off-by: Claudio Fahey --- .../example/streamprocessing/ReaderGroupPruner.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java index 54e59c1f..195bf4f5 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/ReaderGroupPruner.java @@ -84,7 +84,13 @@ protected void doStart() { // We must ensure that we add this reader to the membership synchronizer before the reader group. membershipSynchronizer.startAsync(); membershipSynchronizer.awaitRunning(); - task = executor.scheduleAtFixedRate(new PruneRunner(), heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); + // Initial delay will be between 50% and 100% of the heartbeat interval. + // Although this is not needed for correctness, it reduces unnecessary load + // caused by multiple processes attempting to put the same reader offline. + final long initialDelay = (long) (heartbeatIntervalMillis * (0.5 + Math.random() * 0.5)); + // Period will between 90% and 100% of the heartbeat interval. + final long period = (long) (heartbeatIntervalMillis * (0.9 + Math.random() * 0.1)); + task = executor.scheduleAtFixedRate(new PruneRunner(), initialDelay, period, TimeUnit.MILLISECONDS); notifyStarted(); } From f329c6b7efd17713998d085a5ac8a26f33b66a44 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 21 Aug 2020 05:06:10 +0000 Subject: [PATCH 57/78] Revert log level to ERROR Signed-off-by: Claudio Fahey --- pravega-client-examples/src/main/resources/logback.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/resources/logback.xml b/pravega-client-examples/src/main/resources/logback.xml index 34f9c7c0..ae1a2a69 100644 --- a/pravega-client-examples/src/main/resources/logback.xml +++ b/pravega-client-examples/src/main/resources/logback.xml @@ -18,7 +18,7 @@ - + From c3eae9167401bb85fb1e8d2945216729360504e2 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 12 Sep 2020 02:10:17 +0000 Subject: [PATCH 58/78] Move inline class definition to top-level class SampleEventProcessor Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 35 ++-------- .../SampleEventProcessor.java | 67 +++++++++++++++++++ 2 files changed, 71 insertions(+), 31 deletions(-) create mode 100644 pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 16aee75a..68fa79ce 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -101,7 +101,7 @@ public void run() throws Exception { new JSONSerializer<>(new TypeToken(){}.getType()), EventWriterConfig.builder().build())) { - final AtLeastOnceProcessor processor = new AtLeastOnceProcessor( + final SampleEventProcessor processor = new SampleEventProcessor( () -> ReaderGroupPruner.create( readerGroup, getConfig().getMembershipSynchronizerStreamName(), @@ -114,36 +114,9 @@ public void run() throws Exception { readerGroup.getGroupName(), new JSONSerializer<>(new TypeToken(){}.getType()), ReaderConfig.builder().build()), - 1000) - { - /** - * Process an event that was read. - * Processing can be performed asynchronously after this method returns. - * This method must be stateless. - * - * For this demonstration, we output the same event that was read but with - * the processedBy field set. - * - * @param eventRead The event read. - */ - @Override - public void process(EventRead eventRead) { - final SampleEvent event = eventRead.getEvent(); - event.processedBy = instanceId; - event.processedLatencyMs = System.currentTimeMillis() - event.timestamp; - log.info("{}", event); - writer.writeEvent(event.routingKey, event); - } - - /** - * If {@link #process} did not completely process prior events, it must do so before returning. - * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. - */ - @Override - public void flush() { - writer.flush(); - } - }; + 1000, + instanceId, + writer); processor.startAsync(); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java new file mode 100644 index 00000000..bc027855 --- /dev/null +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java @@ -0,0 +1,67 @@ +/* + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. + * + * Licensed 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 + * + */ +package io.pravega.example.streamprocessing; + +import io.pravega.client.stream.EventRead; +import io.pravega.client.stream.EventStreamReader; +import io.pravega.client.stream.EventStreamWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Supplier; + +/** + * This is an implementation of AtLeastOnceProcessor that performs a simple + * operation to demonstrate processing a SampleEvent. + * For this demonstration, we output to a Pravega stream the same event that was read but with + * the processedBy field set. + */ +public class SampleEventProcessor extends AtLeastOnceProcessor { + private static final Logger log = LoggerFactory.getLogger(SampleEventProcessor.class); + + private final String instanceId; + private final EventStreamWriter writer; + + public SampleEventProcessor(Supplier prunerSupplier, + Supplier> readerSupplier, + long readTimeoutMillis, + String instanceId, + EventStreamWriter writer) { + super(prunerSupplier, readerSupplier, readTimeoutMillis); + this.instanceId = instanceId; + this.writer = writer; + } + + /** + * Process an event that was read. + * Processing can be performed asynchronously after this method returns. + * This method must be stateless. + * + * @param eventRead The event read. + */ + @Override + public void process(EventRead eventRead) { + final SampleEvent event = eventRead.getEvent(); + event.processedBy = instanceId; + event.processedLatencyMs = System.currentTimeMillis() - event.timestamp; + log.info("{}", event); + writer.writeEvent(event.routingKey, event); + } + + /** + * If {@link #process} did not completely process prior events, it must do so before returning. + * If writing to a Pravega stream, this should call {@link EventStreamWriter#flush}. + */ + @Override + public void flush() { + writer.flush(); + } +} From c80ef0d6189b9c59600bed91b86665be87964d8d Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 12 Sep 2020 02:12:05 +0000 Subject: [PATCH 59/78] Clean up EventDebugSink: Make methods private. Add final. Signed-off-by: Claudio Fahey --- .../io/pravega/example/streamprocessing/EventDebugSink.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 0f26d4a8..f818224f 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -46,11 +46,11 @@ public EventDebugSink(AppConfiguration config) { this.config = config; } - public AppConfiguration getConfig() { + private AppConfiguration getConfig() { return config; } - public void run() throws Exception { + private void run() throws Exception { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { streamManager.createScope(getConfig().getScope()); @@ -79,7 +79,7 @@ public void run() throws Exception { long eventCounter = 0; long sum = 0; for (;;) { - EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + final EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); if (eventRead.getEvent() != null) { eventCounter++; sum += eventRead.getEvent().intData; From ca1d896bb96748cce82508046679775ad98592b5 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 12 Sep 2020 02:36:38 +0000 Subject: [PATCH 60/78] Remove commented code Signed-off-by: Claudio Fahey --- .../streamprocessing/WorkerProcessGroup.java | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index e859669f..ae12ab06 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -48,27 +48,6 @@ public void start(int... instanceIds) { }); } - /** - * Processors are guaranteed to not process events written after this method returns. - */ -// public void pause(int... instanceIds) { -// IntStream.of(instanceIds).parallel().forEach(instanceId -> { -// workers.get(instanceId).pause(); -// }); -// } - -// public void preventFlush(int... instanceIds) { -// IntStream.of(instanceIds).parallel().forEach(instanceId -> { -// workers.get(instanceId).induceFailureDuringFlush(); -// }); -// } - -// public void setWriteModeAll(WriteMode mode) { -// IntStream.of(getInstanceIds()).parallel().forEach(instanceId -> { -// workers.get(instanceId).setWriteMode(mode); -// }); -// } - /** * Processors are guaranteed to not process events after this method returns. */ From 3a9978ad083aa6fa9800e94f1da1466b997ecd0b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 12 Sep 2020 03:18:26 +0000 Subject: [PATCH 61/78] Replace some try-with-resource blocks with Lombok @Cleanup Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 75 +++++++++---------- .../streamprocessing/EventDebugSink.java | 46 ++++++------ .../streamprocessing/EventGenerator.java | 44 +++++------ 3 files changed, 82 insertions(+), 83 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 68fa79ce..e3c4f528 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -25,6 +25,7 @@ import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; +import lombok.Cleanup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -89,48 +90,46 @@ public void run() throws Exception { .stream(Stream.of(getConfig().getScope(), getConfig().getStream1Name())) .automaticCheckpointIntervalMillis(getConfig().getCheckpointPeriodMs()) .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig)) { - // Create the Reader Group (ignored if it already exists) - readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); - final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); - try (EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); - // Create a Pravega stream writer that we will send our processed output to. - EventStreamWriter writer = eventStreamClientFactory.createEventWriter( - getConfig().getStream2Name(), - new JSONSerializer<>(new TypeToken(){}.getType()), - EventWriterConfig.builder().build())) { + @Cleanup ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig); + // Create the Reader Group (ignored if it already exists) + readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); + @Cleanup final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); + @Cleanup final EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + @Cleanup final SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); + // Create a Pravega stream writer that we will send our processed output to. + @Cleanup final EventStreamWriter writer = eventStreamClientFactory.createEventWriter( + getConfig().getStream2Name(), + new JSONSerializer<>(new TypeToken(){}.getType()), + EventWriterConfig.builder().build()); - final SampleEventProcessor processor = new SampleEventProcessor( - () -> ReaderGroupPruner.create( - readerGroup, - getConfig().getMembershipSynchronizerStreamName(), - instanceId, - synchronizerClientFactory, - Executors.newScheduledThreadPool(1), - getConfig().getHeartbeatIntervalMillis()), - () -> eventStreamClientFactory.createReader( - instanceId, - readerGroup.getGroupName(), - new JSONSerializer<>(new TypeToken(){}.getType()), - ReaderConfig.builder().build()), - 1000, + final SampleEventProcessor processor = new SampleEventProcessor( + () -> ReaderGroupPruner.create( + readerGroup, + getConfig().getMembershipSynchronizerStreamName(), instanceId, - writer); + synchronizerClientFactory, + Executors.newScheduledThreadPool(1), + getConfig().getHeartbeatIntervalMillis()), + () -> eventStreamClientFactory.createReader( + instanceId, + readerGroup.getGroupName(), + new JSONSerializer<>(new TypeToken(){}.getType()), + ReaderConfig.builder().build()), + 1000, + instanceId, + writer); - processor.startAsync(); + processor.startAsync(); - // Add shutdown hook for graceful shutdown. - Runtime.getRuntime().addShutdownHook(new Thread(() -> { - log.info("Running shutdown hook."); - processor.stopAsync(); - log.info("Waiting for processor to terminate."); - processor.awaitTerminated(); - log.info("Processor terminated."); - })); + // Add shutdown hook for graceful shutdown. + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + log.info("Running shutdown hook."); + processor.stopAsync(); + log.info("Waiting for processor to terminate."); + processor.awaitTerminated(); + log.info("Processor terminated."); + })); - processor.awaitTerminated(); - } - } + processor.awaitTerminated(); } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index f818224f..49f425e3 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -22,6 +22,7 @@ import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; +import lombok.Cleanup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,31 +69,30 @@ private void run() throws Exception { final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(getConfig().getScope(), getConfig().getStream2Name())) .build(); - try (ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI())) { - readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamReader reader = clientFactory.createReader( - "reader", - readerGroup, - new JSONSerializer<>(new TypeToken(){}.getType()), - ReaderConfig.builder().build())) { - long eventCounter = 0; - long sum = 0; - for (;;) { - final EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - if (eventRead.getEvent() != null) { - eventCounter++; - sum += eventRead.getEvent().intData; - log.info("eventCounter={}, sum={}, event={}", - String.format("%6d", eventCounter), - String.format("%8d", sum), - eventRead.getEvent()); - } + @Cleanup final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI()); + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new JSONSerializer<>(new TypeToken(){}.getType()), + ReaderConfig.builder().build())) { + long eventCounter = 0; + long sum = 0; + for (;;) { + final EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + if (eventRead.getEvent() != null) { + eventCounter++; + sum += eventRead.getEvent().intData; + log.info("eventCounter={}, sum={}, event={}", + String.format("%6d", eventCounter), + String.format("%8d", sum), + eventRead.getEvent()); } } - finally { - readerGroupManager.deleteReaderGroup(readerGroup); - } + } + finally { + readerGroupManager.deleteReaderGroup(readerGroup); } } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 469a72f2..b88c5de2 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -18,6 +18,7 @@ import io.pravega.client.stream.EventWriterConfig; import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.StreamConfiguration; +import lombok.Cleanup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,30 +63,29 @@ public void run() throws Exception { } Random rand = new Random(42); - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamWriter writer = clientFactory.createEventWriter( + @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter( getConfig().getStream1Name(), new JSONSerializer<>(new TypeToken(){}.getType()), - EventWriterConfig.builder().build())) { - long sequenceNumber = 0; - long sum = 0; - final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); - for (;;) { - sequenceNumber++; - final SampleEvent event = new SampleEvent(); - event.sequenceNumber = sequenceNumber; - event.routingKey = String.format("%3d", rand.nextInt(1000)); - event.intData = rand.nextInt(1000); - sum += event.intData; - event.sum = sum; - event.timestamp = System.currentTimeMillis(); - event.timestampStr = dateFormat.format(new Date(event.timestamp)); - log.info("{}", event); - final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); - final long ackedSequenceNumber = sequenceNumber; - writeFuture.thenRun(() -> log.debug("Acknowledged: sequenceNumber={}", ackedSequenceNumber)); - Thread.sleep(1000); - } + EventWriterConfig.builder().build()); + long sequenceNumber = 0; + long sum = 0; + final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); + for (;;) { + sequenceNumber++; + final SampleEvent event = new SampleEvent(); + event.sequenceNumber = sequenceNumber; + event.routingKey = String.format("%3d", rand.nextInt(1000)); + event.intData = rand.nextInt(1000); + sum += event.intData; + event.sum = sum; + event.timestamp = System.currentTimeMillis(); + event.timestampStr = dateFormat.format(new Date(event.timestamp)); + log.info("{}", event); + final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); + final long ackedSequenceNumber = sequenceNumber; + writeFuture.thenRun(() -> log.debug("Acknowledged: sequenceNumber={}", ackedSequenceNumber)); + Thread.sleep(1000); } } From 5c563565436d7901d1d9c2133d7f1d0d53de722a Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 12 Sep 2020 03:18:59 +0000 Subject: [PATCH 62/78] Add final Signed-off-by: Claudio Fahey --- .../io/pravega/example/streamprocessing/EventGenerator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index b88c5de2..0c87fe86 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -62,7 +62,7 @@ public void run() throws Exception { streamManager.updateStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); } - Random rand = new Random(42); + final Random rand = new Random(42); @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter( getConfig().getStream1Name(), From f8155cf826d57d942d2b1003878983fcd2cfa3ea Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 19 Mar 2021 16:05:35 +0000 Subject: [PATCH 63/78] Move annotations to separate lines Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 15 +++++++---- .../streamprocessing/EventDebugSink.java | 3 ++- .../streamprocessing/EventGenerator.java | 6 +++-- .../StreamProcessingTest.java | 27 ++++++++++++------- .../streamprocessing/WorkerProcessConfig.java | 12 ++++++--- 5 files changed, 42 insertions(+), 21 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index e3c4f528..afbceea1 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -90,14 +90,19 @@ public void run() throws Exception { .stream(Stream.of(getConfig().getScope(), getConfig().getStream1Name())) .automaticCheckpointIntervalMillis(getConfig().getCheckpointPeriodMs()) .build(); - @Cleanup ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig); + @Cleanup + ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), clientConfig); // Create the Reader Group (ignored if it already exists) readerGroupManager.createReaderGroup(getConfig().getReaderGroup(), readerGroupConfig); - @Cleanup final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); - @Cleanup final EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - @Cleanup final SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); + @Cleanup + final ReaderGroup readerGroup = readerGroupManager.getReaderGroup(getConfig().getReaderGroup()); + @Cleanup + final EventStreamClientFactory eventStreamClientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + @Cleanup + final SynchronizerClientFactory synchronizerClientFactory = SynchronizerClientFactory.withScope(getConfig().getScope(), clientConfig); // Create a Pravega stream writer that we will send our processed output to. - @Cleanup final EventStreamWriter writer = eventStreamClientFactory.createEventWriter( + @Cleanup + final EventStreamWriter writer = eventStreamClientFactory.createEventWriter( getConfig().getStream2Name(), new JSONSerializer<>(new TypeToken(){}.getType()), EventWriterConfig.builder().build()); diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index 49f425e3..d3b995d4 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -69,7 +69,8 @@ private void run() throws Exception { final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(getConfig().getScope(), getConfig().getStream2Name())) .build(); - @Cleanup final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI()); + @Cleanup + final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI()); readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); EventStreamReader reader = clientFactory.createReader( diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 0c87fe86..1b7a9601 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -63,8 +63,10 @@ public void run() throws Exception { } final Random rand = new Random(42); - @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter( + @Cleanup + final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + @Cleanup + final EventStreamWriter writer = clientFactory.createEventWriter( getConfig().getStream1Name(), new JSONSerializer<>(new TypeToken(){}.getType()), EventWriterConfig.builder().build()); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 6f822346..2932c247 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -145,12 +145,14 @@ public void noProcessorTest() throws Exception { final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); final String inputStreamName = "stream-" + UUID.randomUUID().toString(); SETUP_UTILS.get().createTestStream(inputStreamName, 6); - @Cleanup final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); + @Cleanup + final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(scope, clientConfig); // Prepare writer that will write to the stream. final Serializer serializer = new JSONSerializer<>(new TypeToken() {}.getType()); final EventWriterConfig eventWriterConfig = EventWriterConfig.builder().build(); - @Cleanup final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); + @Cleanup + final EventStreamWriter writer = clientFactory.createEventWriter(inputStreamName, serializer, eventWriterConfig); // Prepare reader that will read from the stream. final String outputStreamName = inputStreamName; @@ -160,9 +162,11 @@ public void noProcessorTest() throws Exception { final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(SETUP_UTILS.get().getStream(outputStreamName)) .build(); - @Cleanup final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); + @Cleanup + final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(scope, clientConfig); readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - @Cleanup final EventStreamReader reader = clientFactory.createReader( + @Cleanup + final EventStreamReader reader = clientFactory.createReader( readerId, readerGroup, new JSONSerializer<>(new TypeToken() {}.getType()), @@ -181,14 +185,19 @@ public void noProcessorTest() throws Exception { @Builder protected static class EndToEndTestConfig { // number of stream segments - @Builder.Default public final int numSegments = 1; + @Builder.Default + public final int numSegments = 1; // number of unique routi - @Builder.Default public final int numKeys = 1; + @Builder.Default + public final int numKeys = 1; // number of initial processor instances - @Builder.Default public final int numInitialInstances = 1; - @Builder.Default public final WriteMode writeMode = WriteMode.Default; + @Builder.Default + public final int numInitialInstances = 1; + @Builder.Default + public final WriteMode writeMode = WriteMode.Default; // test-specific function to write and validate events, etc. - @Builder.Default public final Consumer func = (ctx) -> {}; + @Builder.Default + public final Consumer func = (ctx) -> {}; } /** diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java index 1e282385..16d79b6e 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessConfig.java @@ -22,8 +22,12 @@ public class WorkerProcessConfig { public final String outputStreamName; public final String membershipSynchronizerStreamName; public final int numSegments; - @Builder.Default public final long checkpointPeriodMs = 1000; - @Builder.Default public final long heartbeatIntervalMillis = 1000; - @Builder.Default public final long readTimeoutMillis = 1000; - @Builder.Default public final WriteMode writeMode = WriteMode.Default; + @Builder.Default + public final long checkpointPeriodMs = 1000; + @Builder.Default + public final long heartbeatIntervalMillis = 1000; + @Builder.Default + public final long readTimeoutMillis = 1000; + @Builder.Default + public final WriteMode writeMode = WriteMode.Default; } From 2ed765db69295ad9c2b50888bc0ce0e8780b36a7 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 19 Mar 2021 17:29:57 +0000 Subject: [PATCH 64/78] Fix samples version number and improve feedback in runAtLastOnceApp.sh Signed-off-by: Claudio Fahey --- scripts/runAtLeastOnceApp.sh | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/scripts/runAtLeastOnceApp.sh b/scripts/runAtLeastOnceApp.sh index 10979900..becf6d8b 100755 --- a/scripts/runAtLeastOnceApp.sh +++ b/scripts/runAtLeastOnceApp.sh @@ -2,13 +2,19 @@ set -ex ROOT_DIR=$(dirname $0)/.. cd ${ROOT_DIR} -./gradlew pravega-client-examples:build +./gradlew pravega-client-examples:distTar cd pravega-client-examples/build/distributions -tar -xf pravega-client-examples-0.7.0.tar -cd pravega-client-examples-0.7.0 +tar -xf pravega-client-examples-0.10.0-SNAPSHOT.tar +cd pravega-client-examples-0.10.0-SNAPSHOT NUM_INSTANCES=${1:-1} for i in $(seq -w 01 $NUM_INSTANCES); do - INSTANCE_ID=$i bin/atLeastOnceApp >& /tmp/atLeastOnceApp-$i.log & + LOG_FILE=/tmp/atLeastOnceApp-$i.log + echo Logging to ${LOG_FILE} + INSTANCE_ID=$i bin/atLeastOnceApp >& ${LOG_FILE} & done + +echo Instances have been started as background jobs. + +tail -f /tmp/atLeastOnceApp-*.log From 913ef6ef758c83e8b87c4460cdee2c8ddce763c0 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 19 Mar 2021 17:30:44 +0000 Subject: [PATCH 65/78] Fix typo in streamprocessing/README Signed-off-by: Claudio Fahey --- .../src/main/java/io/pravega/example/streamprocessing/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index c635c9c9..adf6a8a0 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -64,7 +64,7 @@ These examples include: scripts/runAtLeastOnceApp.sh 2 ``` - You may view the log files `tmp/atLeastOnceApp-*.log`. + You may view the log files `/tmp/atLeastOnceApp-*.log`. - Start the event debug sink: ```shell script From ec39792d93a586a8b80bd3a3953cba73cd4ec798 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 19 Mar 2021 17:31:26 +0000 Subject: [PATCH 66/78] Fix formatting Signed-off-by: Claudio Fahey --- .../io/pravega/example/streamprocessing/EventDebugSink.java | 3 +-- .../pravega/example/streamprocessing/StreamProcessingTest.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index d3b995d4..f9df0c7a 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -91,8 +91,7 @@ private void run() throws Exception { eventRead.getEvent()); } } - } - finally { + } finally { readerGroupManager.deleteReaderGroup(readerGroup); } } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 2932c247..febc5225 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -136,8 +136,7 @@ void writeEventsAndValidate(TestContext ctx, int numEvents, int[] expectedInstan */ @Test public void noProcessorTest() throws Exception { - final String methodName = (new Object() { - }).getClass().getEnclosingMethod().getName(); + final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); log.info("Test case: {}", methodName); // Create stream. From 097fb8b9455541f0bc54057749c9ee20a3b1950b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Fri, 19 Mar 2021 17:32:16 +0000 Subject: [PATCH 67/78] Removed all commented and unused code from SetupUtils.java Signed-off-by: Claudio Fahey --- .../java/io/pravega/utils/SetupUtils.java | 73 ------------------- 1 file changed, 73 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java index 9318e7d5..a51713dc 100644 --- a/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java +++ b/pravega-client-examples/src/test/java/io/pravega/utils/SetupUtils.java @@ -12,14 +12,10 @@ import com.google.common.base.Preconditions; import io.pravega.client.ClientConfig; -import io.pravega.client.EventStreamClientFactory; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.ScalingPolicy; import io.pravega.client.stream.Stream; import io.pravega.client.stream.StreamConfiguration; -import io.pravega.client.stream.impl.Controller; -import io.pravega.client.stream.impl.ControllerImpl; -import io.pravega.client.stream.impl.ControllerImplConfig; import io.pravega.common.concurrent.ExecutorServiceHelpers; import io.pravega.local.InProcPravegaCluster; import lombok.Cleanup; @@ -31,12 +27,8 @@ import javax.annotation.concurrent.NotThreadSafe; import java.io.IOException; -import java.io.InputStream; import java.net.ServerSocket; import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardCopyOption; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -62,9 +54,6 @@ public final class SetupUtils { @Setter private boolean enableTls = false; - @Setter - private boolean enableHostNameValidation = false; - private boolean enableRestServer = true; // The test Scope name. @@ -121,36 +110,6 @@ public void stopAllServices() throws Exception { } } - /** - * Get resources as temp file. - * - * @param resourceName Name of the resource. - * - * @return Path of the temp file. - */ - static String getFileFromResource(String resourceName) { - try { - Path tempPath = Files.createTempFile("test-", ".tmp"); - tempPath.toFile().deleteOnExit(); - try (InputStream stream = SetupUtils.class.getClassLoader().getResourceAsStream(resourceName)) { - Files.copy(SetupUtils.class.getClassLoader().getResourceAsStream(resourceName), tempPath, StandardCopyOption.REPLACE_EXISTING); - } - return tempPath.toFile().getAbsolutePath(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - - /** - * Fetch the controller endpoint for this cluster. - * - * @return URI The controller endpoint to connect to this cluster. - */ - public URI getControllerUri() { - return getClientConfig().getControllerURI(); - } - /** * Fetch the client configuration with which to connect to the controller. */ @@ -158,24 +117,6 @@ public ClientConfig getClientConfig() { return this.gateway.getClientConfig(); } - /** - * Create a controller facade for this cluster. - * @return The controller facade, which must be closed by the caller. - */ - public Controller newController() { - ControllerImplConfig config = ControllerImplConfig.builder() - .clientConfig(getClientConfig()) - .build(); - return new ControllerImpl(config, DEFAULT_SCHEDULED_EXECUTOR_SERVICE); - } - - /** - * Create a {@link EventStreamClientFactory} for this cluster and scope. - */ - public EventStreamClientFactory newClientFactory() { - return EventStreamClientFactory.withScope(this.scope, getClientConfig()); - } - /** * Create the test stream. * @@ -261,14 +202,6 @@ public void start() throws Exception { .enableMetrics(false) .enableAuth(enableAuth) .enableTls(enableTls) - //.certFile(getFileFromResource(CERT_FILE)) // pravega #2519 - //.keyFile(getFileFromResource(KEY_FILE)) - //.jksKeyFile(getFileFromResource(STANDALONE_KEYSTORE_FILE)) - //.jksTrustFile(getFileFromResource(STANDALONE_TRUSTSTORE_FILE)) - //.keyPasswordFile(getFileFromResource(STANDALONE_KEYSTORE_PASSWD_FILE)) - //.passwdFile(getFileFromResource(PASSWD_FILE)) - //.userName(PRAVEGA_USERNAME) - //.passwd(PRAVEGA_PASSWORD) .build(); log.info("Done building"); this.inProcPravegaCluster.setControllerPorts(new int[]{controllerPort}); @@ -290,9 +223,6 @@ public ClientConfig getClientConfig() { log.info("Getting client config"); return ClientConfig.builder() .controllerURI(URI.create(inProcPravegaCluster.getControllerURI())) - //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) - //.validateHostName(enableHostNameValidation) - //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) .build(); } } @@ -317,9 +247,6 @@ public void stop() throws Exception { public ClientConfig getClientConfig() { return ClientConfig.builder() .controllerURI(controllerUri) - //.credentials(new DefaultCredentials(PRAVEGA_PASSWORD, PRAVEGA_USERNAME)) - //.validateHostName(enableHostNameValidation) - //.trustStore(getFileFromResource(CLIENT_TRUST_STORE_FILE)) .build(); } } From a06dbc6e99dea6d516a396097e1e78065f8b43ca Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 25 Mar 2021 19:10:36 +0000 Subject: [PATCH 68/78] Decrease heartbeat interval for kill5of6Test to speed up dead reader detection. Signed-off-by: Claudio Fahey --- .../streamprocessing/StreamProcessingTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index febc5225..1222353a 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -193,6 +193,10 @@ protected static class EndToEndTestConfig { @Builder.Default public final int numInitialInstances = 1; @Builder.Default + public final long checkpointPeriodMs = 1000; + @Builder.Default + public final long heartbeatIntervalMillis = 1000; + @Builder.Default public final WriteMode writeMode = WriteMode.Default; // test-specific function to write and validate events, etc. @Builder.Default @@ -215,7 +219,6 @@ private void run(EndToEndTestConfig config) throws Exception { final String outputStreamName = "output-stream-" + UUID.randomUUID().toString(); final String membershipSynchronizerStreamName = "ms-" + UUID.randomUUID().toString(); final String inputStreamReaderGroupName = "rg" + UUID.randomUUID().toString().replace("-", ""); - final long checkpointPeriodMs = 1000; @Cleanup StreamManager streamManager = StreamManager.create(clientConfig); @@ -228,7 +231,8 @@ private void run(EndToEndTestConfig config) throws Exception { .outputStreamName(outputStreamName) .membershipSynchronizerStreamName(membershipSynchronizerStreamName) .numSegments(config.numSegments) - .checkpointPeriodMs(checkpointPeriodMs) + .checkpointPeriodMs(config.checkpointPeriodMs) + .heartbeatIntervalMillis(config.heartbeatIntervalMillis) .writeMode(config.writeMode) .build(); @Cleanup @@ -265,7 +269,7 @@ private void run(EndToEndTestConfig config) throws Exception { EventStreamReaderIterator readerIterator = new EventStreamReaderIterator<>(validationReader, readTimeoutMills); final TestEventGenerator generator = new TestEventGenerator(config.numKeys); final TestEventValidator validator = new TestEventValidator(); - final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup, checkpointPeriodMs); + final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup, config.checkpointPeriodMs); config.func.accept(ctx); log.info("Cleanup"); @@ -517,6 +521,7 @@ public void kill5of6Test() throws Exception { .numSegments(6) .numKeys(24) .numInitialInstances(6) + .heartbeatIntervalMillis(200) .writeMode(WriteMode.Default) .func(ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); From 5c09d55bf69adc3c1f9e0acac3691cde4adb8468 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 25 Mar 2021 19:11:29 +0000 Subject: [PATCH 69/78] Adding logging Signed-off-by: Claudio Fahey --- .../example/streamprocessing/StreamProcessingTest.java | 7 +++++-- .../example/streamprocessing/TestEventValidator.java | 8 +++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 1222353a..377514a7 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -211,7 +211,7 @@ protected static class EndToEndTestConfig { */ private void run(EndToEndTestConfig config) throws Exception { final String methodName = (new Object() {}).getClass().getEnclosingMethod().getName(); - log.info("Test case: {}", methodName); + log.info("Test case: {}: BEGIN", methodName); final String scope = SETUP_UTILS.get().getScope(); final ClientConfig clientConfig = SETUP_UTILS.get().getClientConfig(); @@ -272,7 +272,7 @@ private void run(EndToEndTestConfig config) throws Exception { final TestContext ctx = new TestContext(writer, readerIterator, generator, validator, workerProcessGroup, config.checkpointPeriodMs); config.func.accept(ctx); - log.info("Cleanup"); + log.info("Test case: {}: CLEANUP", methodName); workerProcessGroup.close(); validationReader.close(); readerGroupManager.deleteReaderGroup(inputStreamReaderGroupName); @@ -283,6 +283,7 @@ private void run(EndToEndTestConfig config) throws Exception { streamManager.deleteStream(scope, inputStreamName); streamManager.deleteStream(scope, outputStreamName); streamManager.deleteStream(scope, membershipSynchronizerStreamName); + log.info("Test case: {}: END", methodName); } /** @@ -525,7 +526,9 @@ public void kill5of6Test() throws Exception { .writeMode(WriteMode.Default) .func(ctx -> { writeEventsAndValidate(ctx, 100, new int[]{0, 1, 2, 3, 4, 5}); + log.info("kill5of6Test: PAUSING WORKERS"); IntStream.rangeClosed(0, 4).forEach(i -> ctx.workerProcessGroup.get(i).pause()); + log.info("kill5of6Test: WRITING MORE EVENTS"); writeEventsAndValidate(ctx, 90, new int[]{5}); }).build()); } diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java index ae2950e5..7551faef 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/TestEventValidator.java @@ -62,10 +62,12 @@ public void validate(Iterator events, Map expectedLast } } } - throw new NoMoreEventsException(MessageFormat.format( + final String msg = MessageFormat.format( "No more events but all expected events were not received; " + - "receivedSequenceNumbers={0}, expectedLastSequenceNumbers={1}", - receivedSequenceNumbers, expectedLastSequenceNumbers)); + "receivedSequenceNumbers={0}, expectedLastSequenceNumbers={1}", + receivedSequenceNumbers, expectedLastSequenceNumbers); + log.error(msg); + throw new NoMoreEventsException(msg); } public void clearCounters() { From bd6029cc91990a2dddd489b320b9b3ffd672decd Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 25 Mar 2021 19:15:33 +0000 Subject: [PATCH 70/78] Add retry to killAndRestart1of1ForcingDuplicatesTest Signed-off-by: Claudio Fahey --- .../StreamProcessingTest.java | 61 +++++++++++-------- 1 file changed, 35 insertions(+), 26 deletions(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 377514a7..71398c63 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -484,33 +484,42 @@ public void handleExceptionDuringFlushTest() throws Exception { * This test processes events and then kills the processor before it receives a checkpoint. * This is expected to produce duplicates. */ - @Test(timeout = 2*60*1000) + @Test(timeout = 3*60*1000) public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { - run(EndToEndTestConfig.builder() - .numSegments(1) - .numKeys(1) - .numInitialInstances(1) - .writeMode(WriteMode.AlwaysDurable) - .func(ctx -> { - writeEventsAndValidate(ctx, 100, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. - // This will update the reader group state to indicate that this reader has read up to this point. - sleep(2*ctx.checkpointPeriodMs); - // Write some events that will be processed and durably written. - final int expectedDuplicateEventCount = 3; - writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); - Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); - // Kill the worker instance before it checkpoints and updates the reader group state. - // There is no control mechanism to prevent the checkpoint. - // If a checkpoint occurs here, this test will fail because duplicates will not be produced. - ctx.workerProcessGroup.get(0).pause(); - // Start a new worker instance. It should identify the dead worker and call readerOffline(null). - // The new worker should produce duplicates. - ctx.workerProcessGroup.start(1); - writeEventsAndValidate(ctx, 90, new int[]{1}); - Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); - }).build()); + for (;;) { + try { + run(EndToEndTestConfig.builder() + .numSegments(1) + .numKeys(1) + .numInitialInstances(1) + .writeMode(WriteMode.AlwaysDurable) + .func(ctx -> { + writeEventsAndValidate(ctx, 100, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Wait for a while to ensure that a checkpoint occurs and all events have been flushed. + // This will update the reader group state to indicate that this reader has read up to this point. + sleep(2*ctx.checkpointPeriodMs); + // Write some events that will be processed and durably written. + final int expectedDuplicateEventCount = 3; + writeEventsAndValidate(ctx, expectedDuplicateEventCount, new int[]{0}); + Assert.assertEquals(0, ctx.validator.getDuplicateEventCount()); + // Kill the worker instance before it checkpoints and updates the reader group state. + // There is no control mechanism to prevent the checkpoint. + // If a checkpoint occurs here, this test will fail because duplicates will not be produced. + ctx.workerProcessGroup.get(0).pause(); + // Start a new worker instance. It should identify the dead worker and call readerOffline(null). + // The new worker should produce duplicates. + ctx.workerProcessGroup.start(1); + writeEventsAndValidate(ctx, 90, new int[]{1}); + Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); + }).build()); + break; + } catch (Exception e) { + // This test will occasionally fail because a checkpoint may have occurred. + // Such a failure should be rare so retrying should eventually work. + log.warn("Retrying failed test", e); + } + } } /** From 9215290b586553d7268e4f4f1a358f833c84be4f Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Thu, 25 Mar 2021 19:37:35 +0000 Subject: [PATCH 71/78] Fix retry in killAndRestart1of1ForcingDuplicatesTest Signed-off-by: Claudio Fahey --- .../pravega/example/streamprocessing/StreamProcessingTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java index 71398c63..3ce14471 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/StreamProcessingTest.java @@ -514,7 +514,7 @@ public void killAndRestart1of1ForcingDuplicatesTest() throws Exception { Assert.assertEquals(expectedDuplicateEventCount, ctx.validator.getDuplicateEventCount()); }).build()); break; - } catch (Exception e) { + } catch (AssertionError e) { // This test will occasionally fail because a checkpoint may have occurred. // Such a failure should be rare so retrying should eventually work. log.warn("Retrying failed test", e); From 843d4c21e4aa404ec9b450530ee874149906c767 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Gracia?= Date: Fri, 26 Mar 2021 09:45:07 +0100 Subject: [PATCH 72/78] Fixed Scala version of Spark dependency in Hadoop samples. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Raúl Gracia --- hadoop-connector-examples/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-connector-examples/build.gradle b/hadoop-connector-examples/build.gradle index 17c6b9f1..dacedea3 100644 --- a/hadoop-connector-examples/build.gradle +++ b/hadoop-connector-examples/build.gradle @@ -39,7 +39,7 @@ dependencies { compile "io.pravega:pravega-keycloak-client:${pravegaKeycloakVersion}" compileOnly "org.apache.hadoop:hadoop-common:${hadoopVersion}" compileOnly "org.apache.hadoop:hadoop-mapreduce-client-core:${hadoopVersion}" - compileOnly "org.apache.spark:spark-core_2.11:${sparkVersion}" + compileOnly "org.apache.spark:spark-core_2.12:${sparkVersion}" } shadowJar { From 8c880cd604a6571b8a9268a11d02e560eb04427e Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:25:47 +0000 Subject: [PATCH 73/78] Make methods private Signed-off-by: Claudio Fahey --- .../io/pravega/example/streamprocessing/AtLeastOnceApp.java | 2 +- .../pravega/example/streamprocessing/WorkerProcessGroup.java | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index afbceea1..904407ba 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -59,7 +59,7 @@ public AppConfiguration getConfig() { return config; } - public void run() throws Exception { + private void run() { // Get the provided instanceId that uniquely identifes this instances of AtLeastOnceApp. // It will be randomly generated if not provided by the user. final String instanceId = getConfig().getInstanceId(); diff --git a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java index ae12ab06..d9675495 100644 --- a/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java +++ b/pravega-client-examples/src/test/java/io/pravega/example/streamprocessing/WorkerProcessGroup.java @@ -20,6 +20,9 @@ /** * This manages a group of WorkerProcess instances. + * The testing framework in {@link StreamProcessingTest} uses this class to + * start and stop multiple instances of {@link AtLeastOnceProcessorInstrumented} + * in a single process. */ @Builder public class WorkerProcessGroup implements AutoCloseable { From 4dc1aae76edfb19fb410ac433e460ed748a51921 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:26:36 +0000 Subject: [PATCH 74/78] Create private method createStream Signed-off-by: Claudio Fahey --- .../streamprocessing/EventDebugSink.java | 73 ++++++++++--------- 1 file changed, 40 insertions(+), 33 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java index f9df0c7a..6af158b4 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventDebugSink.java @@ -30,6 +30,7 @@ /** * A simple example that continuously shows the events in a stream. + * Each instance of this class will read the entire stream. */ public class EventDebugSink { private static final Logger log = LoggerFactory.getLogger(EventDebugSink.class); @@ -51,48 +52,54 @@ private AppConfiguration getConfig() { return config; } - private void run() throws Exception { + private void run() { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); - try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { - streamManager.createScope(getConfig().getScope()); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - getConfig().getTargetRateEventsPerSec(), - getConfig().getScaleFactor(), - getConfig().getMinNumSegments())) - .build(); - streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); - } + createStream(); // Create a reader group that begins at the earliest event. final String readerGroup = UUID.randomUUID().toString().replace("-", ""); final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(getConfig().getScope(), getConfig().getStream2Name())) .build(); - @Cleanup - final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI()); - readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); - try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - EventStreamReader reader = clientFactory.createReader( - "reader", - readerGroup, - new JSONSerializer<>(new TypeToken(){}.getType()), - ReaderConfig.builder().build())) { - long eventCounter = 0; - long sum = 0; - for (;;) { - final EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); - if (eventRead.getEvent() != null) { - eventCounter++; - sum += eventRead.getEvent().intData; - log.info("eventCounter={}, sum={}, event={}", - String.format("%6d", eventCounter), - String.format("%8d", sum), - eventRead.getEvent()); + + try (final ReaderGroupManager readerGroupManager = ReaderGroupManager.withScope(getConfig().getScope(), getConfig().getControllerURI())) { + readerGroupManager.createReaderGroup(readerGroup, readerGroupConfig); + try (EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); + EventStreamReader reader = clientFactory.createReader( + "reader", + readerGroup, + new JSONSerializer<>(new TypeToken() {}.getType()), + ReaderConfig.builder().build())) { + long eventCounter = 0; + long sum = 0; + for (; ; ) { + final EventRead eventRead = reader.readNextEvent(READER_TIMEOUT_MS); + if (eventRead.getEvent() != null) { + eventCounter++; + sum += eventRead.getEvent().intData; + log.info("eventCounter={}, sum={}, event={}", + String.format("%6d", eventCounter), + String.format("%8d", sum), + eventRead.getEvent()); + } } + } finally { + // Delete the reader group since it is not intended to be shared with any other instances. + readerGroupManager.deleteReaderGroup(readerGroup); } - } finally { - readerGroupManager.deleteReaderGroup(readerGroup); + } + } + + private void createStream() { + try (final StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { + streamManager.createScope(getConfig().getScope()); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) + .build(); + streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); } } } From 1ad816359ada8c8a983aa208e529a53787e4bcbd Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:28:54 +0000 Subject: [PATCH 75/78] Make SampleEvent immutable. Replace @Cleanup with try-with-resources. Signed-off-by: Claudio Fahey --- .../streamprocessing/EventGenerator.java | 51 +++++++++---------- .../example/streamprocessing/SampleEvent.java | 45 +++++++++++++--- .../SampleEventProcessor.java | 8 +-- 3 files changed, 66 insertions(+), 38 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 1b7a9601..983320b3 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -48,7 +48,7 @@ public AppConfiguration getConfig() { return config; } - public void run() throws Exception { + private void run() throws Exception { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { streamManager.createScope(getConfig().getScope()); @@ -63,31 +63,30 @@ public void run() throws Exception { } final Random rand = new Random(42); - @Cleanup - final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig); - @Cleanup - final EventStreamWriter writer = clientFactory.createEventWriter( - getConfig().getStream1Name(), - new JSONSerializer<>(new TypeToken(){}.getType()), - EventWriterConfig.builder().build()); - long sequenceNumber = 0; - long sum = 0; - final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); - for (;;) { - sequenceNumber++; - final SampleEvent event = new SampleEvent(); - event.sequenceNumber = sequenceNumber; - event.routingKey = String.format("%3d", rand.nextInt(1000)); - event.intData = rand.nextInt(1000); - sum += event.intData; - event.sum = sum; - event.timestamp = System.currentTimeMillis(); - event.timestampStr = dateFormat.format(new Date(event.timestamp)); - log.info("{}", event); - final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); - final long ackedSequenceNumber = sequenceNumber; - writeFuture.thenRun(() -> log.debug("Acknowledged: sequenceNumber={}", ackedSequenceNumber)); - Thread.sleep(1000); + try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig)) { + try (final EventStreamWriter writer = clientFactory.createEventWriter( + getConfig().getStream1Name(), + new JSONSerializer<>(new TypeToken() {}.getType()), + EventWriterConfig.builder().build())) { + long sequenceNumber = 0; + long sum = 0; + for (; ; ) { + sequenceNumber++; + final String routingKey = String.format("%3d", rand.nextInt(1000)); + final int intData = rand.nextInt(1000); + sum += intData; + final SampleEvent event = new SampleEvent( + sequenceNumber, + routingKey, + intData, + sum); + log.info("{}", event); + final CompletableFuture writeFuture = writer.writeEvent(event.routingKey, event); + final long ackedSequenceNumber = sequenceNumber; + writeFuture.thenRun(() -> log.debug("Acknowledged: sequenceNumber={}", ackedSequenceNumber)); + Thread.sleep(1000); + } + } } } diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java index 2eeeb08e..7a8e46d9 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEvent.java @@ -10,15 +10,44 @@ */ package io.pravega.example.streamprocessing; +import java.text.SimpleDateFormat; +import java.util.Date; + public class SampleEvent { - public long sequenceNumber; - public String routingKey; - public long intData; - public long sum; - public long timestamp; - public String timestampStr; - public long processedLatencyMs; - public String processedBy; + final public long sequenceNumber; + final public String routingKey; + final public long intData; + final public long sum; + final public long timestamp; + final public String timestampStr; + final public long processedLatencyMs; + final public String processedBy; + + static final private SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX"); + + /// Create a new SampleEvent that represents an unprocessed event. + public SampleEvent(long sequenceNumber, String routingKey, long intData, long sum) { + this.sequenceNumber = sequenceNumber; + this.routingKey = routingKey; + this.intData = intData; + this.sum = sum; + this.timestamp = System.currentTimeMillis(); + this.timestampStr = dateFormat.format(new Date(this.timestamp)); + this.processedLatencyMs = 0; + this.processedBy = null; + } + + /// Create a new SampleEvent that represents a processed event. + public SampleEvent(SampleEvent event, String processedBy) { + this.sequenceNumber = event.sequenceNumber; + this.routingKey = event.routingKey; + this.intData = event.intData; + this.sum = event.sum; + this.timestamp = event.timestamp; + this.timestampStr = event.timestampStr; + this.processedLatencyMs = System.currentTimeMillis() - event.timestamp; + this.processedBy = processedBy; + } @Override public String toString() { diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java index bc027855..2a6dc717 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/SampleEventProcessor.java @@ -50,10 +50,10 @@ public SampleEventProcessor(Supplier prunerSupplier, @Override public void process(EventRead eventRead) { final SampleEvent event = eventRead.getEvent(); - event.processedBy = instanceId; - event.processedLatencyMs = System.currentTimeMillis() - event.timestamp; - log.info("{}", event); - writer.writeEvent(event.routingKey, event); + // This is where we would do something useful with the event. + final SampleEvent processedEvent = new SampleEvent(event, instanceId); + log.info("{}", processedEvent); + writer.writeEvent(processedEvent.routingKey, processedEvent); } /** From afa51393181039562d6d8c38816442d24aebec56 Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:29:24 +0000 Subject: [PATCH 76/78] Update README Signed-off-by: Claudio Fahey --- .../example/streamprocessing/README.md | 50 ++++++++++++++++--- 1 file changed, 43 insertions(+), 7 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md index adf6a8a0..5e9da864 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/README.md @@ -36,7 +36,7 @@ These examples include: ``` - Start the event generator. - ```shell script + ```shell cd ~/pravega-samples ./gradlew pravega-client-examples:startEventGenerator ``` @@ -45,31 +45,66 @@ These examples include: You can either set them in your shell (`export PRAVEGA_SCOPE=examples`) or use the below syntax. If you are using a non-local Pravega instance, specify the controller as follows: - ```shell script + ```shell PRAVEGA_CONTROLLER=tcp://pravega.example.com:9090 ./gradlew pravega-client-examples:startEventGenerator ``` Multiple parameters can be specified as follows. - ```shell script + ```shell PRAVEGA_SCOPE=examples PRAVEGA_CONTROLLER=tcp://localhost:9090 ./gradlew pravega-client-examples:startEventGenerator ``` See [AppConfiguration.java](AppConfiguration.java) for available parameters. +- You will see log lines showing the generated events, as follows: + ``` + EventGenerator: SampleEvent{sequenceNumber= 144, routingKey=819, intData=788, sum= 71872, timestampStr=2021-03-27T00:42:41.549Z, processedLatencyMs= 0, processedBy=null} + EventGenerator: SampleEvent{sequenceNumber= 145, routingKey=725, intData=590, sum= 72462, timestampStr=2021-03-27T00:42:42.551Z, processedLatencyMs= 0, processedBy=null} + ``` + - In another window, start one or more instances of the stream processor. The `runAtLeastOnceApp.sh` script can be used to run multiple instances concurrently. - ```shell script + ```shell cd ~/pravega-samples scripts/runAtLeastOnceApp.sh 2 ``` You may view the log files `/tmp/atLeastOnceApp-*.log`. -- Start the event debug sink: - ```shell script +- The log file `/tmp/atLeastOnceApp-02.log` will show the output of processor 02, which will process approximately half of the events. + ``` + SampleEventProcessor: SampleEvent{sequenceNumber= 144, routingKey=819, intData=788, sum= 71872, timestampStr=2021-03-27T00:42:41.549Z, processedLatencyMs= 21, processedBy=02} + SampleEventProcessor: SampleEvent{sequenceNumber= 145, routingKey=725, intData=590, sum= 72462, timestampStr=2021-03-27T00:42:42.551Z, processedLatencyMs= 284, processedBy=02} + ``` + +- Start a single instance of the event debug sink: + ```shell ./gradlew pravega-client-examples:startEventDebugSink ``` + +- The event debug sink will read from the sine Pravega stream that the multiple instances of AtLeastOnceApp are writing to. + ``` + EventDebugSink: eventCounter= 134, sum= 63485, event=SampleEvent{sequenceNumber= 144, routingKey=819, intData=788, sum= 71872, timestampStr=2021-03-27T00:42:41.549Z, processedLatencyMs= 21, processedBy=02} + EventDebugSink: eventCounter= 220, sum= 109622, event=SampleEvent{sequenceNumber= 145, routingKey=725, intData=590, sum= 72462, timestampStr=2021-03-27T00:42:42.551Z, processedLatencyMs= 284, processedBy=02} + ``` + +- Go ahead and kill one of the AtLeastOnceApp processes. + ```shell + kill $(jps | grep AtLeastOnceApp | awk '{print $1;}') + ``` + +- The remaining AtLeastOnceApp will immediately begin processing any segments that were assigned to the killed process. + +- Run the kill command again to kill the remaining process. + Now you will see that the event debug sink stops showing new events. + +- Start one new processor with `scripts/runAtLeastOnceApp.sh 1`. + You will see the event debug sink resumes showing processed events, with no gaps. + Since the AtLeastOnceApp was stopped gracefully, there will also be no duplicates. + +- Repeat this experiment with `kill -9` to see how it behaves with an ungraceful shutdown. + You may see duplicates. # Parallelism @@ -162,9 +197,10 @@ stored positions. # Achieving Exactly Once Semantics Exactly-once semantics can be achieved by using an idempotent writer with an at-least-once processor. -The `AtLeastOnceApp` writes its output to a Pravega stream using `EventStreamWriter` which is *not* idempotent. +The `AtLeastOnceApp` writes its output to a non-transactional Pravega stream using `EventStreamWriter` which is *not* idempotent. However, if this were modified to write to a key/value store or relational database with a deterministic key, then the writer would be idempotent and the system would provide exactly-once semantics. +Another technique is to use a transactional Pravega stream, which does offer idempotence. # Stateful Exactly-once Semantics with Apache Flink From 40675a4ee674db64ac2ac328f467908fb7e3bd6b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:43:53 +0000 Subject: [PATCH 77/78] Add createStreams method to AtLeastOnceApp Signed-off-by: Claudio Fahey --- .../streamprocessing/AtLeastOnceApp.java | 45 ++++++++++--------- 1 file changed, 25 insertions(+), 20 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java index 904407ba..6192bf85 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/AtLeastOnceApp.java @@ -45,6 +45,7 @@ public class AtLeastOnceApp { private static final Logger log = LoggerFactory.getLogger(AtLeastOnceApp.class); private final AppConfiguration config; + private final ClientConfig clientConfig; public static void main(String[] args) throws Exception { final AtLeastOnceApp app = new AtLeastOnceApp(new AppConfiguration(args)); @@ -53,6 +54,7 @@ public static void main(String[] args) throws Exception { public AtLeastOnceApp(AppConfiguration config) { this.config = config; + this.clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); } public AppConfiguration getConfig() { @@ -65,26 +67,7 @@ private void run() { final String instanceId = getConfig().getInstanceId(); log.info("instanceId={}", instanceId); - // Define configuration to connect to Pravega. - final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); - - // Create the input and output streams (ignored if they already exist). - try (StreamManager streamManager = StreamManager.create(clientConfig)) { - streamManager.createScope(getConfig().getScope()); - final StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - getConfig().getTargetRateEventsPerSec(), - getConfig().getScaleFactor(), - getConfig().getMinNumSegments())) - .build(); - streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); - streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); - // Create stream for the membership state synchronizer. - streamManager.createStream( - getConfig().getScope(), - getConfig().getMembershipSynchronizerStreamName(), - StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); - } + createStreams(); final ReaderGroupConfig readerGroupConfig = ReaderGroupConfig.builder() .stream(Stream.of(getConfig().getScope(), getConfig().getStream1Name())) @@ -137,4 +120,26 @@ private void run() { processor.awaitTerminated(); } + + /** + * Create the input and output streams (ignored if they already exist). + */ + private void createStreams() { + try (StreamManager streamManager = StreamManager.create(clientConfig)) { + streamManager.createScope(getConfig().getScope()); + final StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) + .build(); + streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); + streamManager.createStream(getConfig().getScope(), getConfig().getStream2Name(), streamConfig); + // Create stream for the membership state synchronizer. + streamManager.createStream( + getConfig().getScope(), + getConfig().getMembershipSynchronizerStreamName(), + StreamConfiguration.builder().scalingPolicy(ScalingPolicy.fixed(1)).build()); + } + } } From f67ea436c7e0bedaff3e2d60669ccd5d4435754b Mon Sep 17 00:00:00 2001 From: Claudio Fahey Date: Sat, 27 Mar 2021 01:53:47 +0000 Subject: [PATCH 78/78] Add createStreams private method to EventGenerator Signed-off-by: Claudio Fahey --- .../streamprocessing/EventGenerator.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java index 983320b3..8f6f97dc 100644 --- a/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java +++ b/pravega-client-examples/src/main/java/io/pravega/example/streamprocessing/EventGenerator.java @@ -50,18 +50,7 @@ public AppConfiguration getConfig() { private void run() throws Exception { final ClientConfig clientConfig = ClientConfig.builder().controllerURI(getConfig().getControllerURI()).build(); - try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { - streamManager.createScope(getConfig().getScope()); - StreamConfiguration streamConfig = StreamConfiguration.builder() - .scalingPolicy(ScalingPolicy.byEventRate( - getConfig().getTargetRateEventsPerSec(), - getConfig().getScaleFactor(), - getConfig().getMinNumSegments())) - .build(); - streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); - streamManager.updateStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); - } - + createStreams(); final Random rand = new Random(42); try (final EventStreamClientFactory clientFactory = EventStreamClientFactory.withScope(getConfig().getScope(), clientConfig)) { try (final EventStreamWriter writer = clientFactory.createEventWriter( @@ -90,4 +79,17 @@ private void run() throws Exception { } } + private void createStreams() { + try (StreamManager streamManager = StreamManager.create(getConfig().getControllerURI())) { + streamManager.createScope(getConfig().getScope()); + StreamConfiguration streamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.byEventRate( + getConfig().getTargetRateEventsPerSec(), + getConfig().getScaleFactor(), + getConfig().getMinNumSegments())) + .build(); + streamManager.createStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); + streamManager.updateStream(getConfig().getScope(), getConfig().getStream1Name(), streamConfig); + } + } }