private JoinInputNode.InputType getInputType(RelNode relNode, TranslatorContext context) { // NOTE: Any intermediate form of a join is always a stream. Eg: For the second level join of // stream-table-table join, the left side of the join is join output, which we always // assume to be a stream. The intermediate stream won't be an instance of EnumerableTableScan. // The join key(s) for the table could be an udf in which case the relNode would be LogicalProject. if (relNode instanceof EnumerableTableScan || relNode instanceof LogicalProject) { SqlIOConfig sourceTableConfig = resolveSourceConfigForTable(relNode, context); if (sourceTableConfig == null || !sourceTableConfig.getTableDescriptor().isPresent()) { return JoinInputNode.InputType.STREAM; } else if (sourceTableConfig.getTableDescriptor().get() instanceof RemoteTableDescriptor || sourceTableConfig.getTableDescriptor().get() instanceof CachingTableDescriptor) { return JoinInputNode.InputType.REMOTE_TABLE; } else { return JoinInputNode.InputType.LOCAL_TABLE; } } else { return JoinInputNode.InputType.STREAM; } }
private boolean isTable(RelNode relNode) { // NOTE: Any intermediate form of a join is always a stream. Eg: For the second level join of // stream-table-table join, the left side of the join is join output, which we always // assume to be a stream. The intermediate stream won't be an instance of EnumerableTableScan. if (relNode instanceof EnumerableTableScan) { return resolveSourceConfig(relNode).getTableDescriptor().isPresent(); } else { return false; } }
private Table loadLocalTable(boolean isTablePosOnRight, List<Integer> tableKeyIds, LogicalJoin join, TranslatorContext context) { RelNode relNode = isTablePosOnRight ? join.getRight() : join.getLeft(); MessageStream<SamzaSqlRelMessage> relOutputStream = context.getMessageStream(relNode.getId()); SqlIOConfig sourceConfig = resolveSourceConfig(relNode); if (!sourceConfig.getTableDescriptor().isPresent()) { String errMsg = "Failed to resolve table source in join operation: node=" + relNode; log.error(errMsg); throw new SamzaException(errMsg); } // Create a table backed by RocksDb store with the fields in the join condition as composite key and relational // message as the value. Send the messages from the input stream denoted as 'table' to the created table store. Table<KV<SamzaSqlCompositeKey, SamzaSqlRelMessage>> table = context.getStreamAppDescriptor().getTable(sourceConfig.getTableDescriptor().get()); relOutputStream .map(m -> new KV(createSamzaSqlCompositeKey(m, tableKeyIds), m)) .sendTo(table); return table; } }
final String source = sqlIOConfig.getSource(); final boolean isRemoteTable = sqlIOConfig.getTableDescriptor().isPresent() && (sqlIOConfig.getTableDescriptor().get() instanceof RemoteTableDescriptor || sqlIOConfig.getTableDescriptor().get() instanceof CachingTableDescriptor);
private void sendToOutputStream(String queryLogicalId, String logicalOpId, String sinkStream, StreamApplicationDescriptor appDesc, TranslatorContext translatorContext, RelNode node, int queryId) { SqlIOConfig sinkConfig = sqlConfig.getOutputSystemStreamConfigsBySource().get(sinkStream); MessageStream<SamzaSqlRelMessage> stream = translatorContext.getMessageStream(node.getId()); MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(queryLogicalId, logicalOpId, sinkStream, queryId)); Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor(); if (!tableDescriptor.isPresent()) { KVSerde<Object, Object> noOpKVSerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()); String systemName = sinkConfig.getSystemName(); DelegatingSystemDescriptor sd = systemDescriptors.computeIfAbsent(systemName, DelegatingSystemDescriptor::new); GenericOutputDescriptor<KV<Object, Object>> osd = sd.getOutputDescriptor(sinkConfig.getStreamId(), noOpKVSerde); OutputStream stm = outputMsgStreams.computeIfAbsent(sinkConfig.getSource(), v -> appDesc.getOutputStream(osd)); outputStream.sendTo(stm); } else { Table outputTable = appDesc.getTable(tableDescriptor.get()); if (outputTable == null) { String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource(); throw new SamzaException(msg); } outputStream.sendTo(outputTable); } } }
MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(targetName, queryId)); Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor(); if (!tableDescriptor.isPresent()) { OutputStream stm = outputMsgStreams.computeIfAbsent(source, v -> streamAppDesc.getOutputStream(osd));
if (sourceTableConfig == null || !sourceTableConfig.getTableDescriptor().isPresent()) { String errMsg = "Failed to resolve table source in join operation: node=" + tableNode.getRelNode(); log.error(errMsg); context.getStreamAppDescriptor().getTable(sourceTableConfig.getTableDescriptor().get());
private void sendToOutputStream(StreamApplicationDescriptor appDesc, TranslatorContext context, RelNode node, int queryId) { SqlIOConfig sinkConfig = sqlConfig.getOutputSystemStreamConfigsBySource().get(SamzaSqlApplicationConfig.SAMZA_SYSTEM_LOG); MessageStream<SamzaSqlRelMessage> stream = context.getMessageStream(node.getId()); MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(SamzaSqlApplicationConfig.SAMZA_SYSTEM_LOG, queryId)); Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor(); if (!tableDescriptor.isPresent()) { KVSerde<Object, Object> noOpKVSerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()); String systemName = sinkConfig.getSystemName(); DelegatingSystemDescriptor sd = systemDescriptors.computeIfAbsent(systemName, DelegatingSystemDescriptor::new); GenericOutputDescriptor<KV<Object, Object>> osd = sd.getOutputDescriptor(sinkConfig.getStreamName(), noOpKVSerde); if (OutputMapFunction.logOutputStream == null) { OutputMapFunction.logOutputStream = appDesc.getOutputStream(osd); } outputStream.sendTo(OutputMapFunction.logOutputStream); } else { Table outputTable = appDesc.getTable(tableDescriptor.get()); if (outputTable == null) { String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource(); throw new SamzaException(msg); } outputStream.sendTo(outputTable); } } }
when(mockIOConfig.getTableDescriptor()).thenReturn(Optional.of(mockTableDesc));