private MessageStream<SamzaSqlRelMessage> translateFlatten(Integer flattenIndex, MessageStream<SamzaSqlRelMessage> inputStream) { return inputStream.flatMap(message -> { Object field = message.getSamzaSqlRelRecord().getFieldValues().get(flattenIndex); if (field != null && field instanceof List) { List<SamzaSqlRelMessage> outMessages = new ArrayList<>(); for (Object fieldValue : (List) field) { List<Object> newValues = new ArrayList<>(message.getSamzaSqlRelRecord().getFieldValues()); newValues.set(flattenIndex, Collections.singletonList(fieldValue)); outMessages.add(new SamzaSqlRelMessage(message.getSamzaSqlRelRecord().getFieldNames(), newValues)); } return outMessages; } else { return Collections.singletonList(message); } }); }
/** * Create composite key from the rel message. * @param message Represents the samza sql rel message to extract the key values and names from. * @param relIdx list of keys in the form of field indices within the rel message. * @return the composite key of the rel message */ public static SamzaSqlRelRecord createSamzaSqlCompositeKey(SamzaSqlRelMessage message, List<Integer> relIdx) { return createSamzaSqlCompositeKey(message, relIdx, getSamzaSqlCompositeKeyFieldNames(message.getSamzaSqlRelRecord().getFieldNames(), relIdx)); }
protected KV<Object, Object> convertToSamzaMessage(SamzaSqlRelMessage relMessage, Schema payloadSchema) { return new KV<>(relMessage.getKey(), convertToGenericRecord(relMessage.getSamzaSqlRelRecord(), payloadSchema)); }
public static SamzaSqlRelRecord getMessageKeyRelRecord(SamzaSqlRelMessage message, List<Integer> streamFieldIds, List<String> tableFieldNames, List<Integer> tableKeyIds) { return createSamzaSqlCompositeKey(message, streamFieldIds, getSamzaSqlCompositeKeyFieldNames(tableFieldNames, tableKeyIds)); }
@Override public SamzaSqlRelMessage apply(SamzaSqlRelMessage message, R record) { if (joinRelType.compareTo(JoinRelType.INNER) == 0 && record == null) { log.debug("Inner Join: Record not found for the message with key: " + getMessageKey(message)); // Returning null would result in Join operator implementation to filter out the message. return null; } // The resulting join output should be a SamzaSqlRelMessage containing the fields from both the stream message and // table record. The order of stream message fields and table record fields are dictated by the position of stream // and table in the 'from' clause of sql query. The output should also include the keys from both the stream message // and the table record. List<Object> outFieldValues = new ArrayList<>(); // If table position is on the right, add the stream message fields first if (isTablePosOnRight) { outFieldValues.addAll(message.getSamzaSqlRelRecord().getFieldValues()); } // Add the table record fields. if (record != null) { outFieldValues.addAll(getTableRelRecordFieldValues(record)); } else { // Table record could be null as the record could not be found in the store. This can // happen for outer joins. Add nulls to all the field values in the output message. tableFieldNames.forEach(s -> outFieldValues.add(null)); } // If table position is on the left, add the stream message fields last if (!isTablePosOnRight) { outFieldValues.addAll(message.getSamzaSqlRelRecord().getFieldValues()); } return new SamzaSqlRelMessage(outFieldNames, outFieldValues, message.getSamzaSqlRelMsgMetadata()); }
@Test public void testWithInnerJoinWithTableOnRight() { SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues, new SamzaSqlRelMsgMetadata("", "", "")); JoinRelType joinRelType = JoinRelType.INNER; List<Integer> streamKeyIds = Arrays.asList(0, 1); List<Integer> tableKeyIds = Arrays.asList(0, 1); SamzaSqlRelRecord compositeKey = SamzaSqlRelMessage.createSamzaSqlCompositeKey(tableMsg, tableKeyIds); KV<SamzaSqlRelRecord, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg); JoinInputNode mockTableInputNode = mock(JoinInputNode.class); when(mockTableInputNode.getKeyIds()).thenReturn(tableKeyIds); when(mockTableInputNode.isPosOnRight()).thenReturn(true); when(mockTableInputNode.getFieldNames()).thenReturn(tableFieldNames); JoinInputNode mockStreamInputNode = mock(JoinInputNode.class); when(mockStreamInputNode.getKeyIds()).thenReturn(streamKeyIds); when(mockStreamInputNode.isPosOnRight()).thenReturn(false); when(mockStreamInputNode.getFieldNames()).thenReturn(streamFieldNames); SamzaSqlLocalTableJoinFunction joinFn = new SamzaSqlLocalTableJoinFunction(mockStreamInputNode, mockTableInputNode, joinRelType); SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record); Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(), outMsg.getSamzaSqlRelRecord().getFieldNames().size()); List<String> expectedFieldNames = new ArrayList<>(streamFieldNames); expectedFieldNames.addAll(tableFieldNames); List<Object> expectedFieldValues = new ArrayList<>(streamFieldValues); expectedFieldValues.addAll(tableFieldValues); Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues); }
@Test (expected = IllegalArgumentException.class) public void testCompositeKeyCreationWithInEqualKeyNameValues() { List<String> keyPartNames = Arrays.asList("kfield1", "kfield2"); SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelRecord relRecord1 = SamzaSqlRelMessage.createSamzaSqlCompositeKey(message, Arrays.asList(1, 0), SamzaSqlRelMessage.getSamzaSqlCompositeKeyFieldNames(keyPartNames, Arrays.asList(1))); } }
/** * Create the SamzaSqlCompositeKey from the rel message. * @param message Represents the samza sql rel message. * @param relIdx list of keys in the form of field indices within the rel message. * @return the composite key of the rel message */ public static SamzaSqlCompositeKey createSamzaSqlCompositeKey(SamzaSqlRelMessage message, List<Integer> relIdx) { ArrayList<Object> keyParts = new ArrayList<>(); for (int idx : relIdx) { keyParts.add(message.getSamzaSqlRelRecord().getFieldValues().get(idx)); } return new SamzaSqlCompositeKey(keyParts); } }
@Test public void testEquality() { SamzaSqlRelMessage message1 = new SamzaSqlRelMessage(names, values, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelMessage message2 = new SamzaSqlRelMessage(Arrays.asList("field1", "field2"), Arrays.asList("value1", "value2"), new SamzaSqlRelMsgMetadata("", "", "")); Assert.assertEquals(message1, message2); Assert.assertEquals(message1.hashCode(), message2.hashCode()); }
/** * Converts the nested avro object in SamzaMessage to relational message corresponding to * the tableName with relational schema. */ @Override public SamzaSqlRelMessage convertToRelMessage(KV<Object, Object> samzaMessage) { List<String> payloadFieldNames = new ArrayList<>(); List<Object> payloadFieldValues = new ArrayList<>(); Object value = samzaMessage.getValue(); if (value instanceof IndexedRecord) { fetchFieldNamesAndValuesFromIndexedRecord((IndexedRecord) value, payloadFieldNames, payloadFieldValues, payloadSchema); } else if (value == null) { // If the payload is null, set each record value as null. payloadFieldNames.addAll(payloadSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList())); IntStream.range(0, payloadFieldNames.size()).forEach(x -> payloadFieldValues.add(null)); } else { String msg = "Avro message converter doesn't support messages of type " + value.getClass(); LOG.error(msg); throw new SamzaException(msg); } return new SamzaSqlRelMessage(samzaMessage.getKey(), payloadFieldNames, payloadFieldValues, new SamzaSqlRelMsgMetadata("", "", "")); }
Assert.assertEquals(message.getSamzaSqlRelRecord().getFieldNames().size(), ComplexRecord.SCHEMA$.getFields().size() + 1); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("id").get(), id); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("bool_value").get(), boolValue); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("double_value").get(), doubleValue); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("string_value").get(), new Utf8(testStrValue)); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("float_value").get(), doubleValue); Assert.assertEquals(message.getSamzaSqlRelRecord().getField("long_value").get(), longValue); Assert.assertTrue( arrayValue.stream() .map(Utf8::new) .collect(Collectors.toList()) .equals(message.getSamzaSqlRelRecord().getField("array_values").get())); Assert.assertTrue(mapValue.entrySet() .stream() .collect(Collectors.toMap(x -> new Utf8(x.getKey()), y -> new Utf8(y.getValue()))) .equals(message.getSamzaSqlRelRecord().getField("map_values").get())); Arrays.equals(((ByteString) message.getSamzaSqlRelRecord().getField("bytes_value").get()).getBytes(), testBytes.array())); Assert.assertTrue( Arrays.equals(((ByteString) message.getSamzaSqlRelRecord().getField("fixed_value").get()).getBytes(), DEFAULT_TRACKING_ID_BYTES)); LOG.info(message.toString());
@Override public SamzaSqlRelMessage apply(SamzaSqlInputMessage samzaSqlInputMessage) { Instant startProcessing = Instant.now(); /* SAMZA-2089/LISAMZA-10654: the SamzaRelConverter.convertToRelMessage currently does not initialize * the samzaSqlRelMessage.samzaSqlRelMsgMetadata, this needs to be fixed */ SamzaSqlRelMessage retMsg = this.msgConverter.convertToRelMessage(samzaSqlInputMessage.getKeyAndMessageKV()); retMsg.setEventTime(samzaSqlInputMessage.getMetadata().getEventTime()); retMsg.setArrivalTime(samzaSqlInputMessage.getMetadata().getarrivalTime()); retMsg.setScanTime(startProcessing.toString()); updateMetrics(startProcessing, Instant.now()); return retMsg; }
.partitionBy(m -> createSamzaSqlCompositeKey(m, tableKeyIds), m -> m, KVSerde.of(keySerde, valueSerde), intermediateStreamPrefix + "table_" + logicalOpId) .sendTo(table);
/** * update metrics given a message * @param message the input message * @return the same message */ @Override public SamzaSqlRelMessage apply(SamzaSqlRelMessage message) { inputEvents.inc(); message.getSamzaSqlRelMsgMetadata().operatorBeginProcessingInstant = Instant.now().toString(); return message; }
/** * transforms the input message into the output message with projected fields * @param message the input message to be transformed * @return the new SamzaSqlRelMessage message */ @Override public SamzaSqlRelMessage apply(SamzaSqlRelMessage message) { Instant arrivalTime = Instant.now(); RelDataType type = project.getRowType(); Object[] output = new Object[type.getFieldCount()]; expr.execute(translatorContext.getExecutionContext(), translatorContext.getDataContext(), message.getSamzaSqlRelRecord().getFieldValues().toArray(), output); List<String> names = new ArrayList<>(); for (int index = 0; index < output.length; index++) { names.add(index, project.getNamedProjects().get(index).getValue()); } updateMetrics(arrivalTime, Instant.now(), message.getSamzaSqlRelMsgMetadata().isNewInputMessage); return new SamzaSqlRelMessage(names, Arrays.asList(output), message.getSamzaSqlRelMsgMetadata()); }
@Test public void testWithInnerJoinWithTableOnLeft() { SamzaSqlRelMessage streamMsg = new SamzaSqlRelMessage(streamFieldNames, streamFieldValues, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelMessage tableMsg = new SamzaSqlRelMessage(tableFieldNames, tableFieldValues, new SamzaSqlRelMsgMetadata("", "", "")); JoinRelType joinRelType = JoinRelType.INNER; List<Integer> streamKeyIds = Arrays.asList(0, 2); List<Integer> tableKeyIds = Arrays.asList(0, 2); SamzaSqlRelRecord compositeKey = SamzaSqlRelMessage.createSamzaSqlCompositeKey(tableMsg, tableKeyIds); KV<SamzaSqlRelRecord, SamzaSqlRelMessage> record = KV.of(compositeKey, tableMsg); JoinInputNode mockTableInputNode = mock(JoinInputNode.class); when(mockTableInputNode.getKeyIds()).thenReturn(tableKeyIds); when(mockTableInputNode.isPosOnRight()).thenReturn(false); when(mockTableInputNode.getFieldNames()).thenReturn(tableFieldNames); JoinInputNode mockStreamInputNode = mock(JoinInputNode.class); when(mockStreamInputNode.getKeyIds()).thenReturn(streamKeyIds); when(mockStreamInputNode.isPosOnRight()).thenReturn(true); when(mockStreamInputNode.getFieldNames()).thenReturn(streamFieldNames); SamzaSqlLocalTableJoinFunction joinFn = new SamzaSqlLocalTableJoinFunction(mockStreamInputNode, mockTableInputNode, joinRelType); SamzaSqlRelMessage outMsg = joinFn.apply(streamMsg, record); Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues().size(), outMsg.getSamzaSqlRelRecord().getFieldNames().size()); List<String> expectedFieldNames = new ArrayList<>(tableFieldNames); expectedFieldNames.addAll(streamFieldNames); List<Object> expectedFieldValues = new ArrayList<>(tableFieldValues); expectedFieldValues.addAll(streamFieldValues); Assert.assertEquals(outMsg.getSamzaSqlRelRecord().getFieldValues(), expectedFieldValues); }
@Test public void testCompositeKeyCreation() { List<String> keyPartNames = Arrays.asList("kfield1", "kfield2"); SamzaSqlRelMessage message = new SamzaSqlRelMessage(names, values, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelRecord relRecord1 = SamzaSqlRelMessage.createSamzaSqlCompositeKey(message, Collections.singletonList(0)); Assert.assertEquals(relRecord1.getFieldNames().size(), 1); Assert.assertEquals(relRecord1.getFieldNames().get(0), "field1"); Assert.assertEquals(relRecord1.getFieldValues().get(0), "value1"); SamzaSqlRelRecord relRecord2 = SamzaSqlRelMessage.createSamzaSqlCompositeKey(message, Arrays.asList(1, 0), SamzaSqlRelMessage.getSamzaSqlCompositeKeyFieldNames(keyPartNames, Arrays.asList(1, 0))); Assert.assertEquals(relRecord2.getFieldNames().size(), 2); Assert.assertEquals(relRecord2.getFieldNames().get(0), "kfield2"); Assert.assertEquals(relRecord2.getFieldValues().get(0), "value2"); Assert.assertEquals(relRecord2.getFieldNames().get(1), "kfield1"); Assert.assertEquals(relRecord2.getFieldValues().get(1), "value1"); }
@Override protected List<Object> getTableRelRecordFieldValues(KV record) { // Using the message rel converter, convert message to sql rel message and add to output values. SamzaSqlRelMessage relMessage = msgConverter.convertToRelMessage(record); return relMessage.getSamzaSqlRelRecord().getFieldValues(); }
@Test public void testInEquality() { SamzaSqlRelMessage message1 = new SamzaSqlRelMessage(names, values, new SamzaSqlRelMsgMetadata("", "", "")); SamzaSqlRelMessage message2 = new SamzaSqlRelMessage(Arrays.asList("field1", "field2"), Arrays.asList("value2", "value2"), new SamzaSqlRelMsgMetadata("", "", "")); Assert.assertNotEquals(message1, message2); Assert.assertNotEquals(message1.hashCode(), message2.hashCode()); }
inputStream .map(inputMetricsMF) .partitionBy(m -> createSamzaSqlCompositeKey(m, streamKeyIds, getSamzaSqlCompositeKeyFieldNames(tableFieldNames, tableKeyIds)), m -> m, KVSerde.of(keySerde, valueSerde), intermediateStreamPrefix + "stream_" + logicalOpId) .map(KV::getValue)