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 String apply(FieldInfo fieldInfo) { return fieldInfo.name(); } }
@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); }
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); }
@Override public String apply(FieldInfo fieldInfo) { return fieldInfo.name(); } }