@Test(expected = StreamValidationException.class) public void testStartFailsOnTopicValidationErrors() { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); // create an admin that throws an exception during validateStream SystemAdmin mockAdmin = newAdmin("0", "10"); doThrow(new StreamValidationException("invalid stream")).when(mockAdmin).validateStream(checkpointSpec); SystemFactory factory = newFactory(mock(SystemProducer.class), mock(SystemConsumer.class), mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, true, mock(Config.class), mock(MetricsRegistry.class), null, new KafkaCheckpointLogKeySerde()); // expect an exception during startup checkpointManager.createResources(); checkpointManager.start(); }
@Test(expected = SamzaException.class) public void testReadFailsOnSerdeExceptions() throws Exception { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); Config mockConfig = mock(Config.class); when(mockConfig.get(JobConfig.SSP_GROUPER_FACTORY())).thenReturn(GROUPER_FACTORY_CLASS); // mock out a consumer that returns a single checkpoint IME SystemStreamPartition ssp = new SystemStreamPartition("system-1", "input-topic", new Partition(0)); List<List<IncomingMessageEnvelope>> checkpointEnvelopes = ImmutableList.of( ImmutableList.of(newCheckpointEnvelope(TASK1, ssp, "0"))); SystemConsumer mockConsumer = newConsumer(checkpointEnvelopes); SystemAdmin mockAdmin = newAdmin("0", "1"); SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); // wire up an exception throwing serde with the checkpointmanager KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, true, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointSerde(), new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); // expect an exception from ExceptionThrowingSerde checkpointManager.readLastCheckpoint(TASK1); }
@Test public void testAllMessagesInTheLogAreRead() throws Exception { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); Config mockConfig = mock(Config.class); when(mockConfig.get(JobConfig.SSP_GROUPER_FACTORY())).thenReturn(GROUPER_FACTORY_CLASS); SystemStreamPartition ssp = new SystemStreamPartition("system-1", "input-topic", new Partition(0)); int oldestOffset = 0; int newestOffset = 10; // mock out a consumer that returns ten checkpoint IMEs for the same ssp List<List<IncomingMessageEnvelope>> pollOutputs = new ArrayList<>(); for(int offset = oldestOffset; offset <= newestOffset; offset++) { pollOutputs.add(ImmutableList.of(newCheckpointEnvelope(TASK1, ssp, Integer.toString(offset)))); } // return one message at a time from each poll simulating a KafkaConsumer with max.poll.records = 1 SystemConsumer mockConsumer = newConsumer(pollOutputs); SystemAdmin mockAdmin = newAdmin(Integer.toString(oldestOffset), Integer.toString(newestOffset)); SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, true, mockConfig, mock(MetricsRegistry.class), new CheckpointSerde(), new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); // check that all ten messages are read, and the checkpoint is the newest message Checkpoint checkpoint = checkpointManager.readLastCheckpoint(TASK1); Assert.assertEquals(checkpoint.getOffsets(), ImmutableMap.of(ssp, Integer.toString(newestOffset))); }
@Test public void testCheckpointsAreReadFromOldestOffset() throws Exception { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); Config mockConfig = mock(Config.class); when(mockConfig.get(JobConfig.SSP_GROUPER_FACTORY())).thenReturn(GROUPER_FACTORY_CLASS); // mock out a consumer that returns a single checkpoint IME SystemStreamPartition ssp = new SystemStreamPartition("system-1", "input-topic", new Partition(0)); SystemConsumer mockConsumer = newConsumer(ImmutableList.of( ImmutableList.of(newCheckpointEnvelope(TASK1, ssp, "0")))); String oldestOffset = "0"; SystemAdmin mockAdmin = newAdmin(oldestOffset, "1"); SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, true, mockConfig, mock(MetricsRegistry.class), new CheckpointSerde(), new KafkaCheckpointLogKeySerde()); checkpointManager.register(TASK1); // 1. verify that consumer.register is called only during checkpointManager.start. // 2. verify that consumer.register is called with the oldest offset. // 3. verify that no other operation on the CheckpointManager re-invokes register since start offsets are set during // register verify(mockConsumer, times(0)).register(CHECKPOINT_SSP, oldestOffset); checkpointManager.start(); verify(mockConsumer, times(1)).register(CHECKPOINT_SSP, oldestOffset); checkpointManager.readLastCheckpoint(TASK1); verify(mockConsumer, times(1)).register(CHECKPOINT_SSP, oldestOffset); }
@Test(expected = TopicAlreadyMarkedForDeletionException.class) public void testStartFailsOnTopicCreationErrors() { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); // create an admin that throws an exception during createStream SystemAdmin mockAdmin = newAdmin("0", "10"); doThrow(new TopicAlreadyMarkedForDeletionException("invalid stream")).when(mockAdmin).createStream(checkpointSpec); SystemFactory factory = newFactory(mock(SystemProducer.class), mock(SystemConsumer.class), mockAdmin); KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, true, mock(Config.class), mock(MetricsRegistry.class), null, new KafkaCheckpointLogKeySerde()); // expect an exception during startup checkpointManager.createResources(); checkpointManager.start(); }
@Test public void testReadSucceedsOnKeySerdeExceptionsWhenValidationIsDisabled() throws Exception { KafkaStreamSpec checkpointSpec = new KafkaStreamSpec(CHECKPOINT_TOPIC, CHECKPOINT_TOPIC, CHECKPOINT_SYSTEM, 1); Config mockConfig = mock(Config.class); when(mockConfig.get(JobConfig.SSP_GROUPER_FACTORY())).thenReturn(GROUPER_FACTORY_CLASS); // mock out a consumer that returns a single checkpoint IME SystemStreamPartition ssp = new SystemStreamPartition("system-1", "input-topic", new Partition(0)); List<List<IncomingMessageEnvelope>> checkpointEnvelopes = ImmutableList.of( ImmutableList.of(newCheckpointEnvelope(TASK1, ssp, "0"))); SystemConsumer mockConsumer = newConsumer(checkpointEnvelopes); SystemAdmin mockAdmin = newAdmin("0", "1"); SystemFactory factory = newFactory(mock(SystemProducer.class), mockConsumer, mockAdmin); // wire up an exception throwing serde with the checkpointmanager KafkaCheckpointManager checkpointManager = new KafkaCheckpointManager(checkpointSpec, factory, false, mockConfig, mock(MetricsRegistry.class), new ExceptionThrowingCheckpointSerde(), new ExceptionThrowingCheckpointKeySerde()); checkpointManager.register(TASK1); checkpointManager.start(); // expect the read to succeed inspite of the exception from ExceptionThrowingSerde checkpointManager.readLastCheckpoint(TASK1); }