@SamzaSqlUdfMethod public SamzaSqlRelRecord execute(Object... args) { int numOfArgs = args.length; Validate.isTrue(numOfArgs % 2 == 0, "numOfArgs should be an even number"); List<String> fieldNames = new ArrayList<>(); List<Object> fieldValues = new ArrayList<>(); for (int i = 0; i < numOfArgs - 1; i += 2) { fieldNames.add((String) args[i]); // value can be instanceof SamzaSqlRelRecord, or any Object(string, int, long most likely) fieldValues.add(args[i + 1]); } return new SamzaSqlRelRecord(fieldNames, fieldValues); } }
/** * Creates a {@link SamzaSqlRelMessage} from the list of relational fields and values. * If the field list contains KEY, then it extracts the key out of the fields to create a * {@link SamzaSqlRelRecord} along with key, otherwise creates a {@link SamzaSqlRelRecord} * without the key. * @param fieldNames Ordered list of field names in the row. * @param fieldValues Ordered list of all the values in the row. Some of the fields can be null, This could be * result of delete change capture event in the stream or because of the result of the outer join * or the fields themselves are null in the original stream. */ public SamzaSqlRelMessage(List<String> fieldNames, List<Object> fieldValues) { Validate.isTrue(fieldNames.size() == fieldValues.size(), "Field Names and values are not of same length."); int keyIndex = fieldNames.indexOf(KEY_NAME); Object key = null; if (keyIndex != -1) { key = fieldValues.get(keyIndex); } this.key = key; this.samzaSqlRelRecord = new SamzaSqlRelRecord(fieldNames, fieldValues); }
/** * Create the SamzaSqlRelMessage, Each rel message represents a row in the table. * So it can contain a key and a list of fields in the row. * @param key Represents the key in the row, Key can be null. * @param fieldNames Ordered list of field names in the row. * @param fieldValues Ordered list of all the values in the row. Some of the fields can be null, This could be result of * delete change capture event in the stream or because of the result of the outer join or the fields * themselves are null in the original stream. */ public SamzaSqlRelMessage(Object key, List<String> fieldNames, List<Object> fieldValues) { Validate.isTrue(fieldNames.size() == fieldValues.size(), "Field Names and values are not of same length."); List<String> tmpFieldNames = new ArrayList<>(); List<Object> tmpFieldValues = new ArrayList<>(); this.key = key; tmpFieldNames.add(KEY_NAME); tmpFieldValues.add(key); tmpFieldNames.addAll(fieldNames); tmpFieldValues.addAll(fieldValues); this.samzaSqlRelRecord = new SamzaSqlRelRecord(tmpFieldNames, tmpFieldValues); }
/** * Creates a {@link SamzaSqlRelMessage} from the list of relational fields and values. * If the field list contains KEY, then it extracts the key out of the fields to create a * {@link SamzaSqlRelRecord} along with key, otherwise creates a {@link SamzaSqlRelRecord} * without the key. * @param fieldNames Ordered list of field names in the row. * @param fieldValues Ordered list of all the values in the row. Some of the fields can be null, This could be * result of delete change capture event in the stream or because of the result of the outer join * or the fields themselves are null in the original stream. * @param metadata the message/event's metadata */ public SamzaSqlRelMessage(List<String> fieldNames, List<Object> fieldValues, SamzaSqlRelMsgMetadata metadata) { Validate.isTrue(fieldNames.size() == fieldValues.size(), "Field Names and values are not of same length."); Validate.notNull(metadata, "Message metadata is NULL"); int keyIndex = fieldNames.indexOf(KEY_NAME); Object key = null; if (keyIndex != -1) { key = fieldValues.get(keyIndex); } this.key = key; this.samzaSqlRelRecord = new SamzaSqlRelRecord(fieldNames, fieldValues); this.samzaSqlRelMsgMetadata = metadata; }
/** * Create the SamzaSqlRelMessage, Each rel message represents a row in the table. * So it can contain a key and a list of fields in the row. * @param key Represents the key in the row, Key can be null. * @param fieldNames Ordered list of field names in the row. * @param fieldValues Ordered list of all the values in the row. Some of the fields can be null, This could be result of * delete change capture event in the stream or because of the result of the outer join or the fields * themselves are null in the original stream. * @param metadata the message/event's metadata */ public SamzaSqlRelMessage(Object key, List<String> fieldNames, List<Object> fieldValues, SamzaSqlRelMsgMetadata metadata) { Validate.isTrue(fieldNames.size() == fieldValues.size(), "Field Names and values are not of same length."); Validate.notNull(metadata, "Message metadata is NULL"); List<String> tmpFieldNames = new ArrayList<>(); List<Object> tmpFieldValues = new ArrayList<>(); this.key = key; tmpFieldNames.add(KEY_NAME); tmpFieldValues.add(key); tmpFieldNames.addAll(fieldNames); tmpFieldValues.addAll(fieldValues); this.samzaSqlRelRecord = new SamzaSqlRelRecord(tmpFieldNames, tmpFieldValues); this.samzaSqlRelMsgMetadata = metadata; }
/** * Create composite key from the rel message. * @param message Represents the samza sql rel message to extract the key values from. * @param keyValueIdx list of key values in the form of field indices within the rel message. * @param keyPartNames Represents the key field names. * @return the composite key of the rel message */ public static SamzaSqlRelRecord createSamzaSqlCompositeKey(SamzaSqlRelMessage message, List<Integer> keyValueIdx, List<String> keyPartNames) { Validate.isTrue(keyValueIdx.size() == keyPartNames.size(), "Key part name and value list sizes are different"); ArrayList<Object> keyPartValues = new ArrayList<>(); for (int idx : keyValueIdx) { keyPartValues.add(message.getSamzaSqlRelRecord().getFieldValues().get(idx)); } return new SamzaSqlRelRecord(keyPartNames, keyPartValues); }
private SamzaSqlRelRecord convertToRelRecord(IndexedRecord avroRecord) { List<Object> fieldValues = new ArrayList<>(); List<String> fieldNames = new ArrayList<>(); if (avroRecord != null) { fieldNames.addAll(avroRecord.getSchema().getFields() .stream() .map(Schema.Field::name) .collect(Collectors.toList())); fieldValues.addAll(avroRecord.getSchema().getFields() .stream() .map(f -> convertToJavaObject(avroRecord.get(avroRecord.getSchema().getField(f.name()).pos()), getNonNullUnionSchema(avroRecord.getSchema().getField(f.name()).schema()))) .collect(Collectors.toList())); } else { String msg = "Avro Record is null"; LOG.error(msg); throw new SamzaException(msg); } return new SamzaSqlRelRecord(fieldNames, fieldValues); }
private SamzaSqlRelRecord convertToRelRecord(IndexedRecord avroRecord) { List<Object> values = new ArrayList<>(); List<String> fieldNames = new ArrayList<>(); if (avroRecord != null) { fieldNames.addAll(avroRecord.getSchema().getFields() .stream() .map(Schema.Field::name) .collect(Collectors.toList())); values.addAll(avroRecord.getSchema().getFields() .stream() .map(f -> convertToJavaObject(avroRecord.get(avroRecord.getSchema().getField(f.name()).pos()), getNonNullUnionSchema(avroRecord.getSchema().getField(f.name()).schema()))) .collect(Collectors.toList())); } else { String msg = "Avro Record is null"; LOG.error(msg); throw new SamzaException(msg); } return new SamzaSqlRelRecord(fieldNames, values); }
static Object createRecord(List<String> fieldNames, int level) { String fieldName = fieldNames.get(level); Object child = (level == fieldNames.size() - 1) ? "bar" : createRecord(fieldNames, level + 1); boolean isMap = false; int arrayIndex = -1; if (fieldName.startsWith("map:")) { isMap = true; fieldName = fieldName.substring(4); // strip "map:" } else if (fieldName.endsWith("]")) { arrayIndex = Integer.parseInt(fieldName.substring(fieldName.indexOf("[") + 1, fieldName.length() - 1)); fieldName = fieldName.substring(0, fieldName.indexOf("[")); } if (isMap) { Map<String, Object> retMap = new HashMap<>(); retMap.put(fieldName, child); return retMap; } else if (arrayIndex >= 0) { List list = Arrays.asList(new Object[2 * arrayIndex]); list.set(arrayIndex, child); return list; } else { return new SamzaSqlRelRecord(Collections.singletonList(fieldName), Collections.singletonList(child)); } }
@Test public void testInEquality() { SamzaSqlRelRecord relRecord1 = new SamzaSqlRelRecord(Arrays.asList("id", "name"), Arrays.asList(1L, "object")); SamzaSqlRelRecord relRecord2 = new SamzaSqlRelRecord(Arrays.asList("id", "name"), Arrays.asList(1L, null)); assertNotEquals(relRecord1, relRecord2); assertNotEquals(relRecord1.hashCode(), relRecord2.hashCode()); } }
@Test public void testEquality() { SamzaSqlRelRecord relRecord1 = new SamzaSqlRelRecord(Arrays.asList("id", "name"), Arrays.asList(1L, "object")); SamzaSqlRelRecord relRecord2 = new SamzaSqlRelRecord(Arrays.asList("id", "name"), Arrays.asList(1L, "object")); assertEquals(relRecord1, relRecord2); assertEquals(relRecord1.hashCode(), relRecord2.hashCode()); }
@Test public void testNestedRecord() { BuildOutputRecordUdf buildOutputRecordUdf = new BuildOutputRecordUdf(); SamzaSqlRelRecord nestedSamzaSqlRelRecord = new SamzaSqlRelRecord(Arrays.asList("k3"), Arrays.asList("v3")); SamzaSqlRelRecord actualRecord = buildOutputRecordUdf.execute("k1", "v1", "k2", nestedSamzaSqlRelRecord); SamzaSqlRelRecord expectedRecord = new SamzaSqlRelRecord(Arrays.asList("k1", "k2"), Arrays.asList("v1", nestedSamzaSqlRelRecord)); Assert.assertEquals(actualRecord.getFieldNames(), expectedRecord.getFieldNames()); Assert.assertEquals(actualRecord.getFieldValues(), expectedRecord.getFieldValues()); }
@Test public void testNoArgs() { BuildOutputRecordUdf buildOutputRecordUdf = new BuildOutputRecordUdf(); SamzaSqlRelRecord actualRecord = buildOutputRecordUdf.execute(); SamzaSqlRelRecord expectedRecord = new SamzaSqlRelRecord(new ArrayList<>(), new ArrayList<>()); Assert.assertEquals(actualRecord.getFieldNames(), expectedRecord.getFieldNames()); Assert.assertEquals(actualRecord.getFieldValues(), expectedRecord.getFieldValues()); }
@Test public void testSinglePair() { BuildOutputRecordUdf buildOutputRecordUdf = new BuildOutputRecordUdf(); SamzaSqlRelRecord actualRecord = buildOutputRecordUdf.execute("key", "value"); SamzaSqlRelRecord expectedRecord = new SamzaSqlRelRecord(Arrays.asList("key"), Arrays.asList("value")); Assert.assertEquals(actualRecord.getFieldNames(), expectedRecord.getFieldNames()); Assert.assertEquals(actualRecord.getFieldValues(), expectedRecord.getFieldValues()); }
@Test public void testMultiPairs() { BuildOutputRecordUdf buildOutputRecordUdf = new BuildOutputRecordUdf(); SamzaSqlRelRecord actualRecord = buildOutputRecordUdf.execute("k1", "v1", "k2", "v2"); SamzaSqlRelRecord expectedRecord = new SamzaSqlRelRecord(Arrays.asList("k1", "k2"), Arrays.asList("v1", "v2")); Assert.assertEquals(actualRecord.getFieldNames(), expectedRecord.getFieldNames()); Assert.assertEquals(actualRecord.getFieldValues(), expectedRecord.getFieldValues()); }