@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 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); }