@Override public String apply(FieldInfo fieldInfo) { return fieldInfo.name(); } }
private FieldInfo findPrimaryKeyField(List<FieldInfo> fields) { FieldInfo pkField = null; for (FieldInfo field : fields) { if (field.isPrimary()) { // TODO: this assumes key is only from the one field // if not we need to have order of fields in PK pkField = field; break; } } return pkField; }
@Override public ByteBuffer getMessageFromTuple(Tuple tuple) { return serializer.write((Values) tuple.getValue(1), null); } }
@Override public ISqlStreamsDataSource constructStreams(URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) { List<String> fieldNames = new ArrayList<>(); int primaryIndex = -1; for (int i = 0; i < fields.size(); ++i) { FieldInfo f = fields.get(i); fieldNames.add(f.name()); if (f.isPrimary()) { primaryIndex = i; } } Preconditions.checkState(primaryIndex != -1, "Kafka stream table must have a primary key"); Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames); Map<String, String> values = parseUriParams(uri.getQuery()); String bootstrapServers = values.get(URI_PARAMS_BOOTSTRAP_SERVERS); Preconditions.checkNotNull(bootstrapServers, "bootstrap-servers must be specified"); String topic = uri.getHost(); KafkaSpoutConfig<ByteBuffer, ByteBuffer> kafkaSpoutConfig = new KafkaSpoutConfig.Builder<ByteBuffer, ByteBuffer>(bootstrapServers, topic) .setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteBufferDeserializer.class) .setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteBufferDeserializer.class) .setProp(ConsumerConfig.GROUP_ID_CONFIG, "storm-sql-kafka-" + UUID.randomUUID().toString()) .setRecordTranslator(new RecordTranslatorSchemeAdapter(scheme)) .build(); IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames); return new KafkaStreamsDataSource(kafkaSpoutConfig, bootstrapServers, topic, properties, serializer); }
@Override public void streamsPlan(StreamsPlanCreator planCreator) throws Exception { String sourceName = Joiner.on('.').join(getTable().getQualifiedName()); Map<String, ISqlStreamsDataSource> sources = planCreator.getSources(); if (!sources.containsKey(sourceName)) { throw new RuntimeException("Cannot find table " + sourceName); } List<String> fieldNames = getRowType().getFieldNames(); final Stream<Values> finalStream = planCreator.getStreamBuilder() .newStream(sources.get(sourceName).getProducer(), new StreamsScanTupleValueMapper(fieldNames), parallelismHint); planCreator.addStream(finalStream); } }
@Override public void streamsPlan(StreamsPlanCreator planCreator) throws Exception { // SingleRel RelNode input = getInput(); StormRelUtils.getStormRelInput(input).streamsPlan(planCreator); Stream<Values> inputStream = planCreator.pop(); Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported."); // Calcite ensures that the value is structurized to the table definition // hence we can use PK index directly // To elaborate, if table BAR is defined as ID INTEGER PK, NAME VARCHAR, DEPTID INTEGER // and query like INSERT INTO BAR SELECT NAME, ID FROM FOO is executed, // Calcite makes the projection ($1 <- ID, $0 <- NAME, null) to the value before INSERT. String tableName = Joiner.on('.').join(getTable().getQualifiedName()); IRichBolt consumer = planCreator.getSources().get(tableName).getConsumer(); // To make logic simple, it assumes that all the tables have one PK (which it should be extended to support composed key), // and provides PairStream(KeyedStream) to consumer bolt. inputStream.mapToPair(new StreamInsertMapToPairFunction(primaryKeyIndex)).to(consumer); planCreator.addStream(inputStream); }
public void interpretCreateTable(SqlCreateTable n) { CompilerUtil.TableBuilderInfo builder = new CompilerUtil.TableBuilderInfo(typeFactory); List<FieldInfo> fields = new ArrayList<>(); for (ColumnDefinition col : n.fieldList()) { builder.field(col.name(), col.type(), col.constraint()); RelDataType dataType = col.type().deriveType(typeFactory); Class<?> javaType = (Class<?>) typeFactory.getJavaClass(dataType); ColumnConstraint constraint = col.constraint(); boolean isPrimary = constraint != null && constraint instanceof ColumnConstraint.PrimaryKey; fields.add(new FieldInfo(col.name(), javaType, isPrimary)); } if (n.parallelism() != null) { builder.parallelismHint(n.parallelism()); } Table table = builder.build(); schema.add(n.tableName(), table); ISqlStreamsDataSource ds = DataSourcesRegistry.constructStreamsDataSource(n.location(), n .inputFormatClass(), n.outputFormatClass(), n.properties(), fields); if (ds == null) { throw new RuntimeException("Failed to find data source for " + n .tableName() + " URI: " + n.location()); } else if (dataSources.containsKey(n.tableName())) { throw new RuntimeException("Duplicated definition for table " + n .tableName()); } dataSources.put(n.tableName(), ds); }
@AfterClass public static void staticCleanup() { DataSourcesRegistry.providerMap().remove("mock"); DataSourcesRegistry.providerMap().remove("mocknested"); DataSourcesRegistry.providerMap().remove("mockgroup"); DataSourcesRegistry.providerMap().remove("mockemp"); DataSourcesRegistry.providerMap().remove("mockdept"); if (cluster != null) { cluster.shutdown(); cluster = null; } }
/** * Construct a streams data source. * @param uri data source uri * @param inputFormatClass input format class * @param outputFormatClass output format class * @param properties Properties * @param fields fields info list * @return StreamsDataSource object */ public static ISqlStreamsDataSource constructStreamsDataSource( URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) { DataSourcesProvider provider = providers.get(uri.getScheme()); if (provider == null) { return null; } return provider.constructStreams(uri, inputFormatClass, outputFormatClass, properties, fields); }
@Override public void setSource(Object source) { next.setSource(source); } }
@Override public void dataReceived(ChannelContext ctx, Values data) { ctx.emit(data); } };
public static ChannelContext chain( ChannelContext next, ChannelHandler handler) { return new ChannelContextAdapter(next, handler); }
@Override public void flush(ChannelContext ctx) { ctx.flush(); }
@Override public byte[] format(Tuple tuple) { //TODO we should handle '\n'. ref DelimitedRecordFormat Values values = (Values) tuple.getValue(1); return serializer.write(values, null).array(); }
@Override public String apply(FieldInfo fieldInfo) { return fieldInfo.name(); } }
@BeforeClass public static void staticSetup() throws Exception { DataSourcesRegistry.providerMap().put("mock", new MockDataSourceProvider()); DataSourcesRegistry.providerMap().put("mocknested", new MockNestedDataSourceProvider()); DataSourcesRegistry.providerMap().put("mockgroup", new MockGroupDataSourceProvider()); DataSourcesRegistry.providerMap().put("mockemp", new MockEmpDataSourceProvider()); DataSourcesRegistry.providerMap().put("mockdept", new MockDeptDataSourceProvider()); cluster = new LocalCluster(); }
@Override public String getValueFromTuple(ITuple tuple) { Values values = (Values) tuple.getValue(1); byte[] array = outputSerializer.write(values, null).array(); return new String(array); } }
@Override public Document toDocument(ITuple tuple) { Document document = new Document(); Values values = (Values) tuple.getValue(1); byte[] array = serializer.write(values, null).array(); document.append(serField, array); return document; }
@Override public void execute(Tuple input) { Values values = (Values) input.getValue(0); byte[] array = serializer.write(values, null).array(); String data = new String(array); try { writer.write(data + "\n"); writer.flush(); collector.ack(input); } catch (IOException e) { LOG.error("Error while writing data to socket.", e); collector.reportError(e); collector.fail(input); } }