/** * Handles the produce request from {@link InMemorySystemProducer} and populates the underlying message queue. * * @param ssp system stream partition * @param key key for message produced * @param message actual payload */ void put(SystemStreamPartition ssp, Object key, Object message) { List<IncomingMessageEnvelope> messages = bufferedMessages.get(ssp); String offset = String.valueOf(messages.size()); if (message instanceof EndOfStreamMessage) { offset = IncomingMessageEnvelope.END_OF_STREAM_OFFSET; } IncomingMessageEnvelope messageEnvelope = new IncomingMessageEnvelope(ssp, offset, key, message); bufferedMessages.get(ssp) .add(messageEnvelope); }
@Override public void complete() { if (scheduledFuture != null) { scheduledFuture.cancel(true); } log.trace("Callback complete for task {}, ssp {}, offset {}.", new Object[] {taskName, envelope.getSystemStreamPartition(), envelope.getOffset()}); if (isComplete.compareAndSet(false, true)) { listener.onComplete(this); } else { String msg = String.format("Callback complete was invoked after completion for task %s, ssp %s, offset %s.", taskName, envelope.getSystemStreamPartition(), envelope.getOffset()); listener.onFailure(this, new IllegalStateException(msg)); } }
@Override public Collection<Object> handleMessage(IncomingMessageEnvelope ime, MessageCollector collector, TaskCoordinator coordinator) { Object message; InputTransformer transformer = inputOpSpec.getTransformer(); if (transformer != null) { message = transformer.apply(ime); } else { message = this.inputOpSpec.isKeyed() ? KV.of(ime.getKey(), ime.getMessage()) : ime.getMessage(); } return Collections.singletonList(message); }
public IncomingMessageEnvelope readNext() { if (!hasNext()) { LOG.warn("Attempting to read more data when there aren't any. ssp=" + systemStreamPartition); return null; } // record the next offset before we read, so when the read fails and we reconnect, // we seek to the same offset that we try below curSingleFileOffset = curReader.nextOffset(); IncomingMessageEnvelope messageEnvelope = curReader.readNext(); // Copy everything except for the offset. Turn the single-file style offset into a multi-file one return new IncomingMessageEnvelope(messageEnvelope.getSystemStreamPartition(), getCurOffset(), messageEnvelope.getKey(), messageEnvelope.getMessage(), messageEnvelope.getSize(), messageEnvelope.getEventTime(), messageEnvelope.getArrivalTime()); }
@Override public Object apply(IncomingMessageEnvelope ime) { Assert.notNull(ime, "ime is null"); KV<Object, Object> keyAndMessageKV = KV.of(ime.getKey(), ime.getMessage()); SamzaSqlRelMsgMetadata metadata = new SamzaSqlRelMsgMetadata(Instant.ofEpochMilli(ime.getEventTime()).toString(), Instant.ofEpochMilli(ime.getArrivalTime()).toString(), null); SamzaSqlInputMessage samzaMsg = SamzaSqlInputMessage.of(keyAndMessageKV, metadata); return samzaMsg; } }
@Test public void testUpdateCallbackInOrder() { TaskName taskName = new TaskName("Partition 0"); SystemStreamPartition ssp = new SystemStreamPartition("kafka", "topic", new Partition(0)); ReadableCoordinator coordinator = new ReadableCoordinator(taskName); IncomingMessageEnvelope envelope0 = new IncomingMessageEnvelope(ssp, "0", null, null); TaskCallbackImpl callback0 = new TaskCallbackImpl(listener, taskName, envelope0, coordinator, 0, 0); List<TaskCallbackImpl> callbacksToUpdate = callbackManager.updateCallback(callback0); assertEquals(1, callbacksToUpdate.size()); TaskCallbackImpl callback = callbacksToUpdate.get(0); assertTrue(callback.matchSeqNum(0)); assertEquals(ssp, callback.envelope.getSystemStreamPartition()); assertEquals("0", callback.envelope.getOffset()); IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp, "1", null, null); TaskCallbackImpl callback1 = new TaskCallbackImpl(listener, taskName, envelope1, coordinator, 1, 0); callbacksToUpdate = callbackManager.updateCallback(callback1); assertEquals(1, callbacksToUpdate.size()); callback = callbacksToUpdate.get(0); assertTrue(callback.matchSeqNum(1)); assertEquals(ssp, callback.envelope.getSystemStreamPartition()); assertEquals("1", callback.envelope.getOffset()); }
private void verifyEvents(List<IncomingMessageEnvelope> messages, List<EventData> eventDataList, Interceptor interceptor) { Assert.assertEquals(messages.size(), eventDataList.size()); for (int i = 0; i < messages.size(); i++) { IncomingMessageEnvelope message = messages.get(i); EventData eventData = eventDataList.get(i); Assert.assertEquals(message.getKey(), eventData.getSystemProperties().getPartitionKey()); Assert.assertEquals(message.getMessage(), interceptor.intercept(eventData.getBytes())); Assert.assertEquals(message.getOffset(), eventData.getSystemProperties().getOffset()); } }
private boolean checkEndOfStream() { if (pendingEnvelopeQueue.size() == 1) { PendingEnvelope pendingEnvelope = pendingEnvelopeQueue.peek(); IncomingMessageEnvelope envelope = pendingEnvelope.envelope; if (envelope.isEndOfStream()) { SystemStreamPartition ssp = envelope.getSystemStreamPartition(); processingSspSet.remove(ssp); if (!hasIntermediateStreams) { pendingEnvelopeQueue.remove(); } } } return processingSspSet.isEmpty(); }
@Test public void testSequentialRead() throws Exception { SystemStreamPartition ssp = new SystemStreamPartition("hdfs", "testStream", new Partition(0)); MultiFileHdfsReader multiReader = new MultiFileHdfsReader(HdfsReaderFactory.ReaderType.AVRO, ssp, Arrays.asList(descriptors), "0:0"); int index = 0; while (multiReader.hasNext()) { GenericRecord record = (GenericRecord) multiReader.readNext().getMessage(); Assert.assertEquals(index % NUM_EVENTS, record.get(FIELD_1)); Assert.assertEquals("string_" + (index % NUM_EVENTS), record.get(FIELD_2).toString()); index++; } Assert.assertEquals(3 * NUM_EVENTS, index); multiReader.close(); }
@Test public void testRandomRead() throws Exception { SystemStreamPartition ssp = new SystemStreamPartition("hdfs", "testStream", new Partition(0)); SingleFileHdfsReader reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, "0"); for (int i = 0;i < NUM_EVENTS / 2; i++) { reader.readNext(); } String offset = reader.nextOffset(); IncomingMessageEnvelope envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record1 = (GenericRecord) envelope.getMessage(); for (int i = 0; i < 5; i++) reader.readNext(); // seek to the offset within the same reader reader.seek(offset); Assert.assertEquals(offset, reader.nextOffset()); envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record2 = (GenericRecord) envelope.getMessage(); Assert.assertEquals(record1, record2); reader.close(); // open a new reader and initialize it with the offset reader = new AvroFileHdfsReader(ssp); reader.open(AVRO_FILE, offset); envelope = reader.readNext(); Assert.assertEquals(offset, envelope.getOffset()); GenericRecord record3 = (GenericRecord) envelope.getMessage(); Assert.assertEquals(record1, record3); reader.close(); }
private void verifyRecords(List<IncomingMessageEnvelope> outputRecords, List<Record> inputRecords, String shardId) { Iterator outputRecordsIter = outputRecords.iterator(); inputRecords.forEach(record -> { IncomingMessageEnvelope envelope = (IncomingMessageEnvelope) outputRecordsIter.next(); String outputKey = (String) envelope.getKey(); KinesisIncomingMessageEnvelope kinesisMessageEnvelope = (KinesisIncomingMessageEnvelope) envelope; Assert.assertEquals(outputKey, record.getPartitionKey()); Assert.assertEquals(kinesisMessageEnvelope.getSequenceNumber(), record.getSequenceNumber()); Assert.assertEquals(kinesisMessageEnvelope.getApproximateArrivalTimestamp(), record.getApproximateArrivalTimestamp()); Assert.assertEquals(kinesisMessageEnvelope.getShardId(), shardId); ByteBuffer outputData = ByteBuffer.wrap((byte[]) kinesisMessageEnvelope.getMessage()); record.getData().rewind(); Assert.assertTrue(outputData.equals(record.getData())); verifyOffset(envelope.getOffset(), record, shardId); }); }
SystemStream systemStream = ime.getSystemStreamPartition().getSystemStream(); InputOperatorImpl inputOpImpl = operatorImplGraph.getInputOperator(systemStream); if (inputOpImpl != null) { switch (MessageType.of(ime.getMessage())) { case USER_MESSAGE: inputOpImpl.onMessage(ime, collector, coordinator); EndOfStreamMessage eosMessage = (EndOfStreamMessage) ime.getMessage(); inputOpImpl.aggregateEndOfStream(eosMessage, ime.getSystemStreamPartition(), collector, coordinator); break; WatermarkMessage watermarkMessage = (WatermarkMessage) ime.getMessage(); inputOpImpl.aggregateWatermark(watermarkMessage, ime.getSystemStreamPartition(), collector, coordinator); break;
assertFalse(endOfStreamStates.allEndOfStream()); IncomingMessageEnvelope envelope = IncomingMessageEnvelope.buildEndOfStreamEnvelope(inputPartition0); endOfStreamStates.update((EndOfStreamMessage) envelope.getMessage(), envelope.getSystemStreamPartition()); assertTrue(endOfStreamStates.isEndOfStream(input)); assertFalse(endOfStreamStates.isEndOfStream(intermediate));
private void moveMessagesToTheirQueue(SystemStreamPartition ssp, List<IncomingMessageEnvelope> envelopes) { long nextOffset = nextOffsets.get(ssp); for (IncomingMessageEnvelope env : envelopes) { sink.addMessage(ssp, env); // move message to the BlockingEnvelopeMap's queue LOG.trace("IncomingMessageEnvelope. got envelope with offset:{} for ssp={}", env.getOffset(), ssp); nextOffset = Long.valueOf(env.getOffset()) + 1; } nextOffsets.put(ssp, nextOffset); }
@Override public Map<SystemStreamPartition, List<IncomingMessageEnvelope>> poll(Set<SystemStreamPartition> set, long timeout) throws InterruptedException { Map<SystemStreamPartition, List<IncomingMessageEnvelope>> envelopeMap = new HashMap<>(); set.forEach(ssp -> { int curMessages = curMessagesPerSsp.get(ssp); // We send num Messages and an end of stream message following that. List<IncomingMessageEnvelope> envelopes = IntStream.range(curMessages, curMessages + numMessages/4) .mapToObj(i -> i < numMessages ? new IncomingMessageEnvelope(ssp, null, getKey(i, ssp), getData(i, ssp)) : IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp)) .collect(Collectors.toList()); envelopeMap.put(ssp, envelopes); curMessagesPerSsp.put(ssp, curMessages + numMessages/4); }); if (sleepBetweenPollsMs > 0) { Thread.sleep(sleepBetweenPollsMs); } return envelopeMap; }
/** * Insert the envelope into the task pending queues and run all the tasks */ private void runTasks(IncomingMessageEnvelope envelope) { if (envelope != null) { PendingEnvelope pendingEnvelope = new PendingEnvelope(envelope); for (AsyncTaskWorker worker : sspToTaskWorkerMapping.get(envelope.getSystemStreamPartition())) { worker.state.insertEnvelope(pendingEnvelope); } } for (AsyncTaskWorker worker: taskWorkers) { worker.run(); } }
@Override public IncomingMessageEnvelope next() { refresh(); if (peeks.size() == 0) { throw new NoSuchElementException(); } IncomingMessageEnvelope envelope = peeks.poll(); if (envelope.isEndOfStream()) { endOfStreamReached = true; } return envelope; }
IncomingMessageEnvelope envelope2 = new IncomingMessageEnvelope(ssp, "2", null, null); TaskCallbackImpl callback2 = new TaskCallbackImpl(listener, taskName, envelope2, coordinator, 2, 0); List<TaskCallbackImpl> callbacksToUpdate = callbackManager.updateCallback(callback2); assertTrue(callbacksToUpdate.isEmpty()); IncomingMessageEnvelope envelope1 = new IncomingMessageEnvelope(ssp, "1", null, null); TaskCallbackImpl callback1 = new TaskCallbackImpl(listener, taskName, envelope1, coordinator, 1, 0); callbacksToUpdate = callbackManager.updateCallback(callback1); assertTrue(callbacksToUpdate.isEmpty()); IncomingMessageEnvelope envelope0 = new IncomingMessageEnvelope(ssp, "0", null, null); TaskCallbackImpl callback0 = new TaskCallbackImpl(listener, taskName, envelope0, coordinator, 0, 0); callbacksToUpdate = callbackManager.updateCallback(callback0); TaskCallbackImpl callback = callbacksToUpdate.get(0); assertTrue(callback.matchSeqNum(0)); assertEquals(ssp, callback.envelope.getSystemStreamPartition()); assertEquals("0", callback.envelope.getOffset()); assertEquals(ssp, callback.envelope.getSystemStreamPartition()); assertEquals("1", callback.envelope.getOffset()); assertEquals(ssp, callback.envelope.getSystemStreamPartition()); assertEquals("2", callback.envelope.getOffset());
private boolean checkEndOfStream() { if (pendingEnvelopeQueue.size() == 1) { PendingEnvelope pendingEnvelope = pendingEnvelopeQueue.peek(); IncomingMessageEnvelope envelope = pendingEnvelope.envelope; if (envelope.isEndOfStream()) { SystemStreamPartition ssp = envelope.getSystemStreamPartition(); processingSspSet.remove(ssp); if (!hasIntermediateStreams) { pendingEnvelopeQueue.remove(); } } } return processingSspSet.isEmpty(); }
@Override public void process( IncomingMessageEnvelope incomingMessageEnvelope, MessageCollector messageCollector, TaskCoordinator taskCoordinator) throws Exception { messageCollector.send( new OutgoingMessageEnvelope( new SystemStream(outputSystem, outputTopic), incomingMessageEnvelope.getMessage())); processedMessageCount++; if (processedMessageCount == expectedMessageCount) { taskCoordinator.shutdown(TaskCoordinator.RequestScope.ALL_TASKS_IN_CONTAINER); } } }