/** * Converts any StreamSpec to a KafkaStreamSpec. * If the original spec already is a KafkaStreamSpec, it is simply returned. * * @param originalSpec The StreamSpec instance to convert to KafkaStreamSpec. * @return A KafkaStreamSpec instance. */ public static KafkaStreamSpec fromSpec(StreamSpec originalSpec) { if (originalSpec instanceof KafkaStreamSpec) { return ((KafkaStreamSpec) originalSpec); } int replicationFactor = Integer.parseInt(originalSpec.getOrDefault( KafkaConfig.TOPIC_REPLICATION_FACTOR(), KafkaConfig.TOPIC_DEFAULT_REPLICATION_FACTOR())); return new KafkaStreamSpec( originalSpec.getId(), originalSpec.getPhysicalName(), originalSpec.getSystemName(), originalSpec.getPartitionCount(), replicationFactor, mapToProperties(filterUnsupportedProperties(originalSpec.getConfig()))); }
/** * Copies this StreamSpec, but applies a new partitionCount. * * This method is not static s.t. subclasses can override it. * * @param partitionCount The partitionCount for the returned StreamSpec. * @return A copy of this StreamSpec with the specified partitionCount. */ public StreamSpec copyWithPartitionCount(int partitionCount) { return new StreamSpec(id, physicalName, systemName, partitionCount, config); }
/** * Populate the metadata for the {@link SystemStream} and initialize the buffer for {@link SystemStreamPartition}. * * @param streamSpec stream spec for the stream to be initialized * * @return true if successful, false otherwise */ boolean initializeStream(StreamSpec streamSpec) { LOG.info("Initializing the stream for {}", streamSpec.getId()); systemStreamToPartitions.put(streamSpec.toSystemStream(), streamSpec.getPartitionCount()); for (int partition = 0; partition < streamSpec.getPartitionCount(); partition++) { bufferedMessages.put( new SystemStreamPartition(streamSpec.toSystemStream(), new Partition(partition)), newSynchronizedLinkedList()); } return true; }
StreamSpec getStreamSpec() { StreamSpec spec = (partitions == PARTITIONS_UNKNOWN) ? streamSpec : streamSpec.copyWithPartitionCount(partitions); if (isIntermediate) { String physicalName = StreamManager.createUniqueNameForBatch(spec.getPhysicalName(), config); if (!physicalName.equals(spec.getPhysicalName())) { spec = spec.copyWithPhysicalName(physicalName); } } return spec; }
@Override public boolean clearStream(StreamSpec streamSpec) { LOG.info("Creating Kafka topic: {} on system: {}", streamSpec.getPhysicalName(), streamSpec.getSystemName()); KafkaSystemAdminUtilsScala.clearStream(streamSpec, getZkConnection()); Map<String, List<PartitionInfo>> topicsMetadata = getTopicMetadata(ImmutableSet.of(streamSpec.getPhysicalName())); return topicsMetadata.get(streamSpec.getPhysicalName()).isEmpty(); }
public void createStreams(List<StreamSpec> streams) { Multimap<String, StreamSpec> streamsGroupedBySystem = HashMultimap.create(); streams.forEach(streamSpec -> streamsGroupedBySystem.put(streamSpec.getSystemName(), streamSpec)); for (Map.Entry<String, Collection<StreamSpec>> entry : streamsGroupedBySystem.asMap().entrySet()) { String systemName = entry.getKey(); SystemAdmin systemAdmin = systemAdmins.getSystemAdmin(systemName); for (StreamSpec stream : entry.getValue()) { LOGGER.info("Creating stream {} with partitions {} on system {}", new Object[]{stream.getPhysicalName(), stream.getPartitionCount(), systemName}); systemAdmin.createStream(stream); } } }
Config generateConfig() { Map<String, String> streamConfig = new HashMap<>(); StreamSpec spec = getStreamSpec(); streamConfig.put(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), spec.getId()), spec.getSystemName()); streamConfig.put(String.format(StreamConfig.PHYSICAL_NAME_FOR_STREAM_ID(), spec.getId()), spec.getPhysicalName()); if (isIntermediate()) { streamConfig.put(String.format(StreamConfig.IS_INTERMEDIATE_FOR_STREAM_ID(), spec.getId()), "true"); streamConfig.put(String.format(StreamConfig.DELETE_COMMITTED_MESSAGES_FOR_STREAM_ID(), spec.getId()), "true"); streamConfig.put(String.format(StreamConfig.CONSUMER_OFFSET_DEFAULT_FOR_STREAM_ID(), spec.getId()), "oldest"); streamConfig.put(String.format(StreamConfig.PRIORITY_FOR_STREAM_ID(), spec.getId()), String.valueOf(Integer.MAX_VALUE)); } spec.getConfig().forEach((property, value) -> { streamConfig.put(String.format(StreamConfig.STREAM_ID_PREFIX(), spec.getId()) + property, value); }); return new MapConfig(streamConfig); }
.map(id -> new StreamSpec(id, streamConfig.getPhysicalName(id), streamConfig.getSystem(id))) .collect(Collectors.toSet()); intStreams.forEach(stream -> { LOGGER.info("Clear intermediate stream {} in system {}", stream.getPhysicalName(), stream.getSystemName()); systemAdmins.getSystemAdmin(stream.getSystemName()).clearStream(stream); }); LOGGER.info("Clear store {} changelog {}", store, changelog); SystemStream systemStream = StreamUtil.getSystemStreamFromNames(changelog); StreamSpec spec = StreamSpec.createChangeLogStreamSpec(systemStream.getStream(), systemStream.getSystem(), 1); systemAdmins.getSystemAdmin(spec.getSystemName()).clearStream(spec);
@Test public void testBasicConstructor() { StreamSpec streamSpec = new StreamSpec("dummyId", "dummyPhysicalName", "dummySystemName", 1); assertEquals("dummyId", streamSpec.getId()); assertEquals("dummyPhysicalName", streamSpec.getPhysicalName()); assertEquals("dummySystemName", streamSpec.getSystemName()); assertEquals(1, streamSpec.getPartitionCount()); // SystemStream should use the physical name, not the streamId. SystemStream systemStream = new SystemStream("dummySystemName", "dummyPhysicalName"); assertEquals(systemStream, streamSpec.toSystemStream()); }
private String streamSpecToJson(StreamSpec streamSpec) { return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(), streamSpec.getPhysicalName()); } }
public void testCreateChangelogStreamHelp(final String topic) { final int PARTITIONS = 12; final int REP_FACTOR = 2; Map<String, String> map = new HashMap<>(); map.put(JobConfig.JOB_DEFAULT_SYSTEM(), SYSTEM()); map.put(String.format("stores.%s.changelog", "fakeStore"), topic); map.put(String.format("stores.%s.changelog.replication.factor", "fakeStore"), String.valueOf(REP_FACTOR)); map.put(String.format("stores.%s.changelog.kafka.segment.bytes", "fakeStore"), "139"); KafkaSystemAdmin admin = Mockito.spy(createSystemAdmin(SYSTEM(), map)); StreamSpec spec = StreamSpec.createChangeLogStreamSpec(topic, SYSTEM(), PARTITIONS); Mockito.doAnswer(invocationOnMock -> { StreamSpec internalSpec = (StreamSpec) invocationOnMock.callRealMethod(); assertTrue(internalSpec instanceof KafkaStreamSpec); // KafkaStreamSpec is used to carry replication factor assertTrue(internalSpec.isChangeLogStream()); assertEquals(SYSTEM(), internalSpec.getSystemName()); assertEquals(topic, internalSpec.getPhysicalName()); assertEquals(REP_FACTOR, ((KafkaStreamSpec) internalSpec).getReplicationFactor()); assertEquals(PARTITIONS, internalSpec.getPartitionCount()); assertEquals("139", ((KafkaStreamSpec) internalSpec).getProperties().getProperty("segment.bytes")); assertEquals("compact", ((KafkaStreamSpec) internalSpec).getProperties().getProperty("cleanup.policy")); return internalSpec; }).when(admin).toKafkaSpec(Mockito.any()); admin.createStream(spec); admin.validateStream(spec); }
@Override public boolean createStream(StreamSpec streamSpec) { LOG.info("Creating Kafka topic: {} on system: {}", streamSpec.getPhysicalName(), streamSpec.getSystemName()); final String REPL_FACTOR = "replication.factor"; Map<String, String> streamConfig = new HashMap(streamSpec.getConfig());
StreamSpec.createChangeLogStreamSpec(systemStream.getStream(), systemStream.getSystem(), maxChangeLogStreamPartitions); String accesslogStream = storageConfig.getAccessLogStream(systemStream.getStream()); StreamSpec accesslogSpec = new StreamSpec(accesslogStream, accesslogStream, systemStream.getSystem(), maxChangeLogStreamPartitions); systemAdmin.createStream(accesslogSpec); systemAdmin.validateStream(accesslogSpec);
@Override public boolean createStream(StreamSpec streamSpec) { createdStreamName = streamSpec.getPhysicalName(); return true; }
void addOutEdge(StreamEdge out) { outEdges.put(out.getStreamSpec().getId(), out); }
@Test public void testGenerateConfig() { // an example unbounded IO stream StreamSpec spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", Collections.singletonMap("property1", "haha")); StreamEdge edge = new StreamEdge(spec, false, false, new MapConfig()); Config config = edge.generateConfig(); StreamConfig streamConfig = new StreamConfig(config); assertEquals(streamConfig.getSystem(spec.getId()), "system-1"); assertEquals(streamConfig.getPhysicalName(spec.getId()), "physical-stream-1"); assertEquals(streamConfig.getIsIntermediateStream(spec.getId()), false); assertEquals(streamConfig.getStreamProperties(spec.getId()).get("property1"), "haha"); // bounded stream spec = new StreamSpec("stream-1", "physical-stream-1", "system-1", Collections.singletonMap("property1", "haha")); edge = new StreamEdge(spec, false, false, new MapConfig()); config = edge.generateConfig(); streamConfig = new StreamConfig(config); // intermediate stream edge = new StreamEdge(spec, true, false, new MapConfig()); config = edge.generateConfig(); streamConfig = new StreamConfig(config); assertEquals(streamConfig.getIsIntermediateStream(spec.getId()), true); assertEquals(streamConfig.getDefaultStreamOffset(spec.toSystemStream()).get(), "oldest"); assertEquals(streamConfig.getPriority(spec.toSystemStream()), Integer.MAX_VALUE); } }
@Test public void testDefaultPartitions() { Map<String, String> map = new HashMap<>(config); map.put(JobConfig.JOB_INTERMEDIATE_STREAM_PARTITIONS(), String.valueOf(DEFAULT_PARTITIONS)); Config cfg = new MapConfig(map); ExecutionPlanner planner = new ExecutionPlanner(cfg, streamManager); StreamApplicationDescriptorImpl graphSpec = createSimpleGraph(); JobGraph jobGraph = (JobGraph) planner.plan(graphSpec); // Partitions should be the same as input1 jobGraph.getIntermediateStreams().forEach(edge -> { assertTrue(edge.getPartitionCount() == DEFAULT_PARTITIONS); }); }
@Test public void testUnsupportedConfigStrippedFromProperties() { StreamSpec original = new StreamSpec("dummyId","dummyPhysicalName", "dummySystemName", ImmutableMap.of("segment.bytes", "4", "replication.factor", "7")); // First verify the original assertEquals("7", original.get("replication.factor")); assertEquals("4", original.get("segment.bytes")); Map<String, String> config = original.getConfig(); assertEquals("7", config.get("replication.factor")); assertEquals("4", config.get("segment.bytes")); // Now verify the Kafka spec KafkaStreamSpec spec = KafkaStreamSpec.fromSpec(original); assertNull(spec.get("replication.factor")); assertEquals("4", spec.get("segment.bytes")); Properties kafkaProperties = spec.getProperties(); Map<String, String> kafkaConfig = spec.getConfig(); assertNull(kafkaProperties.get("replication.factor")); assertEquals("4", kafkaProperties.get("segment.bytes")); assertNull(kafkaConfig.get("replication.factor")); assertEquals("4", kafkaConfig.get("segment.bytes")); }
@Test(expected = IllegalArgumentException.class) public void testGetStreamWithOutSystemInConfig() { Config config = buildStreamConfig(STREAM_ID, StreamConfig.PHYSICAL_NAME(), TEST_PHYSICAL_NAME); StreamSpec spec = StreamUtil.getStreamSpec(STREAM_ID, new StreamConfig(config)); assertEquals(TEST_SYSTEM, spec.getSystemName()); }
Map<String, String> map = new HashMap<>(); Config config = new MapConfig(map); StreamSpec spec = new StreamSpec("id", topicName, SYSTEM(), defaultPartitionCount, config); spec = StreamSpec.createCoordinatorStreamSpec(topicName, SYSTEM()); kafkaSpec = admin.toKafkaSpec(spec); Assert.assertEquals(coordReplicatonFactor, kafkaSpec.getReplicationFactor()); String.valueOf(changeLogReplicationFactor)); admin = Mockito.spy(createSystemAdmin(SYSTEM(), map)); spec = StreamSpec.createChangeLogStreamSpec(topicName, SYSTEM(), changeLogPartitionFactor); kafkaSpec = admin.toKafkaSpec(spec); Assert.assertEquals(changeLogReplicationFactor, kafkaSpec.getReplicationFactor()); spec = StreamSpec.createChangeLogStreamSpec("anotherTopic", SYSTEM(), changeLogPartitionFactor); kafkaSpec = admin.toKafkaSpec(spec); Assert.fail("toKafkaSpec should've failed for missing topic"); spec = new StreamSpec(interStreamId, topicName, SYSTEM(), defaultPartitionCount, config); kafkaSpec = admin.toKafkaSpec(spec); Assert.assertEquals("v1", kafkaSpec.getProperties().getProperty("p1"));