@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; }
final RelNode node = relRoot.project(); ScanTranslator scanTranslator = new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource(), queryId);
/** * 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); } }); }
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; }
@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()); }
@Test public void testGetInputAndOutputStreamConfigsFanOut() { List<String> sqlStmts = Arrays.asList("Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1", "insert into testavro.Profile select * from testavro.SIMPLE1"); 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(1, inputKeys.size()); Assert.assertTrue(inputKeys.contains("testavro.SIMPLE1")); Assert.assertEquals(2, outputKeys.size()); Assert.assertTrue(outputKeys.contains("testavro.COMPLEX1")); Assert.assertTrue(outputKeys.contains("testavro.Profile")); Assert.assertEquals(2, outputStreamList.size()); Assert.assertEquals("testavro.COMPLEX1", outputStreamList.get(0)); Assert.assertEquals("testavro.Profile", outputStreamList.get(1)); }
@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)); }
final RelNode node = relRoot.project(); ScanTranslator scanTranslator = new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource(), queryId); ModifyTranslator modifyTranslator = new ModifyTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getOutputSystemStreamConfigsBySource(), queryId);
when(mockAppConfig.getInputSystemStreamConfigBySource()).thenReturn(ssConfigBySource);