@Override public Collection<RelRoot> convertDsl(String dsl) { // TODO: Introduce an API to parse a dsl string and return one or more sql statements List<String> sqlStmts = fetchSqlFromConfig(config); List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts); SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(config, queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream) .collect(Collectors.toList()), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toList())); QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(), sqlConfig.getUdfMetadata()); List<RelRoot> relRoots = new LinkedList<>(); for (String sql: sqlStmts) { // we always pass only select query to the planner for samza sql. The reason is that samza sql supports // schema evolution where source and destination could up to an extent have independent schema evolution while // calcite expects strict comformance of the destination schema with that of the fields in the select query. SamzaSqlQueryParser.QueryInfo qinfo = SamzaSqlQueryParser.parseQuery(sql); relRoots.add(planner.plan(qinfo.getSelectQuery())); } return relRoots; }
public SamzaSqlApplicationConfig(Config staticConfig, Set<String> inputSystemStreams, Set<String> outputSystemStreams) { ioResolver = createIOResolver(staticConfig); inputSystemStreamConfigBySource = inputSystemStreams.stream() .collect(Collectors.toMap(Function.identity(), src -> ioResolver.fetchSourceInfo(src))); outputSystemStreamConfigsBySource = outputSystemStreams.stream() .collect(Collectors.toMap(Function.identity(), x -> ioResolver.fetchSinkInfo(x))); systemStreamConfigsBySource = new HashMap<>(inputSystemStreamConfigBySource); systemStreamConfigsBySource.putAll(outputSystemStreamConfigsBySource); Set<SqlIOConfig> systemStreamConfigs = new HashSet<>(systemStreamConfigsBySource.values()); relSchemaProvidersBySource = systemStreamConfigs.stream() .collect(Collectors.toMap(SqlIOConfig::getSource, x -> initializePlugin("RelSchemaProvider", x.getRelSchemaProviderName(), staticConfig, CFG_FMT_REL_SCHEMA_PROVIDER_DOMAIN, (o, c) -> ((RelSchemaProviderFactory) o).create(x.getSystemStream(), c)))); samzaRelConvertersBySource = systemStreamConfigs.stream() .collect(Collectors.toMap(SqlIOConfig::getSource, x -> initializePlugin("SamzaRelConverter", x.getSamzaRelConverterName(), staticConfig, CFG_FMT_SAMZA_REL_CONVERTER_DOMAIN, (o, c) -> ((SamzaRelConverterFactory) o).create(x.getSystemStream(), relSchemaProvidersBySource.get(x.getSource()), c)))); udfResolver = createUdfResolver(staticConfig); udfMetadata = udfResolver.getUdfs(); windowDurationMs = staticConfig.getLong(CFG_GROUPBY_WINDOW_DURATION_MS, DEFAULT_GROUPBY_WINDOW_DURATION_MS); // remove the SqlIOConfigs of outputs whose system is "log" out of systemStreamConfigsBySource outputSystemStreamConfigsBySource.forEach((k, v) -> { if (k.split("\\.")[0].equals(SamzaSqlApplicationConfig.SAMZA_SYSTEM_LOG)) { systemStreamConfigsBySource.remove(k); } }); }
public void translate(RelRoot relRoot, TranslatorContext translatorContext, int queryId) { final SqlIOResolver ioResolver = translatorContext.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver(); final RelNode node = relRoot.project(); ScanTranslator scanTranslator = new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource(), queryId); ModifyTranslator modifyTranslator = new ModifyTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getOutputSystemStreamConfigsBySource(), queryId); sqlConfig.getOutputSystemStreamConfigsBySource().keySet().forEach( key -> { if (key.split("\\.")[0].equals(SamzaSqlApplicationConfig.SAMZA_SYSTEM_LOG)) {
@Override public Collection<RelRoot> convertDsl(String dsl) { // TODO: Introduce an API to parse a dsl string and return one or more sql statements List<String> sqlStmts = fetchSqlFromConfig(config); List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts); SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(config, queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream) .collect(Collectors.toSet()), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet())); QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getSystemStreamConfigsBySource(), sqlConfig.getUdfMetadata()); List<RelRoot> relRoots = new LinkedList<>(); for (String sql: sqlStmts) { // when sql is a query, we only pass the select query to the planner SamzaSqlQueryParser.QueryInfo qinfo = SamzaSqlQueryParser.parseQuery(sql); if (qinfo.getSink().split("\\.")[0].equals(SamzaSqlApplicationConfig.SAMZA_SYSTEM_LOG)) { sql = qinfo.getSelectQuery(); } relRoots.add(planner.plan(sql)); } return relRoots; }
/** * For unit testing only */ @VisibleForTesting void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationDescriptor appDesc, int queryId) { QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(), sqlConfig.getUdfMetadata()); final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery()); SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(sqlConfig); TranslatorContext translatorContext = new TranslatorContext(appDesc, relRoot, executionContext); translate(relRoot, sqlConfig.getOutputSystemStreams().get(queryId), translatorContext, queryId); Map<Integer, TranslatorContext> translatorContexts = new HashMap<>(); translatorContexts.put(queryId, translatorContext.clone()); appDesc.withApplicationTaskContextFactory(new ApplicationTaskContextFactory<SamzaSqlApplicationContext>() { @Override public SamzaSqlApplicationContext create(ExternalContext externalContext, JobContext jobContext, ContainerContext containerContext, TaskContext taskContext, ApplicationContainerContext applicationContainerContext) { return new SamzaSqlApplicationContext(translatorContexts); } }); }
@Test public void testGetInputAndOutputStreamConfigsFanIn() { List<String> sqlStmts = Arrays.asList("Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1", "insert into testavro.COMPLEX1 select * from testavro.SIMPLE2"); Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream) .collect(Collectors.toList()), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toList())); Set<String> inputKeys = samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().keySet(); Set<String> outputKeys = samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().keySet(); List<String> outputStreamList = samzaSqlApplicationConfig.getOutputSystemStreams(); Assert.assertEquals(2, inputKeys.size()); Assert.assertTrue(inputKeys.contains("testavro.SIMPLE1")); Assert.assertTrue(inputKeys.contains("testavro.SIMPLE2")); Assert.assertEquals(1, outputKeys.size()); Assert.assertTrue(outputKeys.contains("testavro.COMPLEX1")); Assert.assertEquals(2, outputStreamList.size()); Assert.assertEquals("testavro.COMPLEX1", outputStreamList.get(0)); Assert.assertEquals("testavro.COMPLEX1", outputStreamList.get(1)); }
private void testWithoutConfigShouldPass(Map<String, String> config, String configKey) { Map<String, String> badConfigs = new HashMap<>(config); badConfigs.remove(configKey); List<String> sqlStmts = fetchSqlFromConfig(badConfigs); List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts); new SamzaSqlApplicationConfig(new MapConfig(badConfigs), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream) .collect(Collectors.toList()), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toList())); }
@Test public void testConfigInit() { Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1"); String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config"); int numUdfs = config.get(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES).split(",").length; List<String> sqlStmts = fetchSqlFromConfig(config); List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts); SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream) .collect(Collectors.toList()), queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toList())); Assert.assertEquals(numUdfs, samzaSqlApplicationConfig.getUdfMetadata().size()); Assert.assertEquals(1, samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().size()); Assert.assertEquals(1, samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().size()); }
/** * For unit testing only */ @VisibleForTesting public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationDescriptor appDesc) { QueryPlanner planner = new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getSystemStreamConfigsBySource(), sqlConfig.getUdfMetadata()); final RelRoot relRoot = planner.plan(queryInfo.getSql()); int queryId = 1; SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(sqlConfig); Map<String, SamzaRelConverter> converters = sqlConfig.getSamzaRelConverters(); TranslatorContext translatorContext = new TranslatorContext(appDesc, relRoot, executionContext, converters); translate(relRoot, translatorContext, queryId); Map<Integer, TranslatorContext> translatorContexts = new HashMap<>(); translatorContexts.put(queryId, translatorContext.clone()); appDesc.withApplicationTaskContextFactory((jobContext, containerContext, taskContext, applicationContainerContext) -> new SamzaSqlApplicationContext(translatorContexts)); }
String sqlJson = SamzaSqlApplicationConfig.serializeSqlStmts(dslStmts); newConfig.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, sqlJson); List<String> outputSystemStreams = new LinkedList<>(); SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, config, inputSystemStreams, outputSystemStreams); SqlIOResolver ioResolver = SamzaSqlApplicationConfig.createIOResolver(config);
SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); new SamzaSqlApplicationConfig(appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); TranslatorContext translatorContext = new TranslatorContext(appDescriptor, relRoot, executionContext); translatorContextMap.put(queryId, translatorContext); queryTranslator.translate(relRoot, sqlConfig.getOutputSystemStreams().get(queryId), translatorContext, queryId); queryId++;
SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); new SamzaSqlApplicationConfig(appDescriptor.getConfig(), inputSystemStreams, outputSystemStreams); Map<String, SamzaRelConverter> converters = sqlConfig.getSamzaRelConverters(); for (RelRoot relRoot : relRoots) { LOG.info("Translating relRoot {} to samza stream graph", relRoot);
@Override public SqlSchema getTableSchema(ExecutionContext context, String tableName) { /** * currently Shell works only for systems that has Avro schemas */ lastErrorMsg = ""; int execId = execIdSeq.incrementAndGet(); Map<String, String> staticConfigs = fetchSamzaSqlConfig(execId, context); Config samzaSqlConfig = new MapConfig(staticConfigs); SqlSchema sqlSchema = null; try { SqlIOResolver ioResolver = SamzaSqlApplicationConfig.createIOResolver(samzaSqlConfig); SqlIOConfig sourceInfo = ioResolver.fetchSourceInfo(tableName); RelSchemaProvider schemaProvider = SamzaSqlApplicationConfig.initializePlugin("RelSchemaProvider", sourceInfo.getRelSchemaProviderName(), samzaSqlConfig, SamzaSqlApplicationConfig.CFG_FMT_REL_SCHEMA_PROVIDER_DOMAIN, (o, c) -> ((RelSchemaProviderFactory) o).create(sourceInfo.getSystemStream(), c)); sqlSchema = schemaProvider.getSqlSchema(); } catch (SamzaException ex) { String msg = "getTableSchema failed with exception "; lastErrorMsg = msg + ex.toString(); LOG.error(msg, ex); } return sqlSchema; }
final RelNode node = relRoot.project(); ScanTranslator scanTranslator = new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource(), queryId);
public static SqlIOResolver createIOResolver(Config config) { String sourceResolveValue = config.get(CFG_IO_RESOLVER); Map<String, String> metadataPrefixProperties = new HashMap<>(); metadataPrefixProperties.put( String.format(CFG_FMT_SOURCE_RESOLVER_DOMAIN, sourceResolveValue) + CFG_METADATA_TOPIC_PREFIX, config.get(CFG_METADATA_TOPIC_PREFIX, DEFAULT_METADATA_TOPIC_PREFIX)); Config newConfig = new MapConfig(Arrays.asList(config, metadataPrefixProperties)); Validate.notEmpty(sourceResolveValue, "ioResolver config is not set or empty"); return initializePlugin("SqlIOResolver", sourceResolveValue, newConfig, CFG_FMT_SOURCE_RESOLVER_DOMAIN, (o, c) -> ((SqlIOResolverFactory) o).create(c, newConfig)); }
public SamzaSqlExecutionContext(SamzaSqlApplicationConfig config) { this.sqlConfig = config; udfMetadata = this.sqlConfig.getUdfMetadata().stream().collect(Collectors.toMap(UdfMetadata::getName, Function.identity())); }
private SqlIOConfig resolveSourceConfigForTable(RelNode relNode, TranslatorContext context) { if (relNode instanceof LogicalProject) { return resolveSourceConfigForTable(((LogicalProject) relNode).getInput(), context); } // We are returning the sourceConfig for the table as null when the table is in another join rather than an output // table, that's because the output of stream-table join is considered a stream. if (relNode.getInputs().size() > 1) { return null; } String sourceName = SqlIOConfig.getSourceFromSourceParts(relNode.getTable().getQualifiedName()); SqlIOConfig sourceConfig = context.getExecutionContext().getSamzaSqlApplicationConfig().getInputSystemStreamConfigBySource().get(sourceName); if (sourceConfig == null) { throw new SamzaException("Unsupported source found in join statement: " + sourceName); } return sourceConfig; }
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); } } }
private UdfResolver createUdfResolver(Map<String, String> config) { String udfResolveValue = config.get(CFG_UDF_RESOLVER); Validate.notEmpty(udfResolveValue, "udfResolver config is not set or empty"); HashMap<String, String> domainConfig = getDomainProperties(config, String.format(CFG_FMT_UDF_RESOLVER_DOMAIN, udfResolveValue), false); Properties props = new Properties(); props.putAll(domainConfig); HashMap<String, String> udfConfig = getDomainProperties(config, CFG_UDF_CONFIG_DOMAIN, false); return new ConfigBasedUdfResolver(props, new MapConfig(udfConfig)); }
public static List<String> fetchSqlFromConfig(Map<String, String> config) { List<String> sql; if (config.containsKey(SamzaSqlApplicationConfig.CFG_SQL_STMT) && StringUtils.isNotBlank(config.get(SamzaSqlApplicationConfig.CFG_SQL_STMT))) { String sqlValue = config.get(SamzaSqlApplicationConfig.CFG_SQL_STMT); sql = Collections.singletonList(sqlValue); } else if (config.containsKey(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON) && StringUtils.isNotBlank(config.get(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON))) { sql = SamzaSqlApplicationConfig.deserializeSqlStmts(config.get(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON)); } else if (config.containsKey(SamzaSqlApplicationConfig.CFG_SQL_FILE)) { String sqlFile = config.get(SamzaSqlApplicationConfig.CFG_SQL_FILE); sql = SqlFileParser.parseSqlFile(sqlFile); } else { String msg = "Config doesn't contain the SQL that needs to be executed."; LOG.error(msg); throw new SamzaException(msg); } return sql; } }