You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2017/04/05 23:19:04 UTC
[01/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Repository: storm
Updated Branches:
refs/heads/master e84b39d1e -> c652d3ffc
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
new file mode 100644
index 0000000..6b3dfc9
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.utils;
+
+import static org.apache.commons.lang.StringUtils.isNotEmpty;
+
+import com.google.common.base.Preconditions;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
+import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
+import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
+import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
+import org.apache.storm.sql.runtime.serde.json.JsonScheme;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
+import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
+import org.apache.storm.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public final class SerdeUtils {
+ public static Scheme getScheme(String inputFormatClass, Properties properties, List<String> fieldNames) {
+ Scheme scheme;
+ if (isNotEmpty(inputFormatClass)) {
+ if (JsonScheme.class.getName().equals(inputFormatClass)) {
+ scheme = new JsonScheme(fieldNames);
+ } else if (TsvScheme.class.getName().equals(inputFormatClass)) {
+ String delimiter = properties.getProperty("input.tsv.delimiter", "\t");
+ scheme = new TsvScheme(fieldNames, delimiter.charAt(0));
+ } else if (CsvScheme.class.getName().equals(inputFormatClass)) {
+ scheme = new CsvScheme(fieldNames);
+ } else if (AvroScheme.class.getName().equals(inputFormatClass)) {
+ String schemaString = properties.getProperty("input.avro.schema");
+ Preconditions.checkArgument(isNotEmpty(schemaString), "input.avro.schema can not be empty");
+ scheme = new AvroScheme(schemaString, fieldNames);
+ } else {
+ scheme = Utils.newInstance(inputFormatClass);
+ }
+ } else {
+ //use JsonScheme as the default scheme
+ scheme = new JsonScheme(fieldNames);
+ }
+ return scheme;
+ }
+
+ public static IOutputSerializer getSerializer(String outputFormatClass, Properties properties, List<String> fieldNames) {
+ IOutputSerializer serializer;
+ if (isNotEmpty(outputFormatClass)) {
+ if (JsonSerializer.class.getName().equals(outputFormatClass)) {
+ serializer = new JsonSerializer(fieldNames);
+ } else if (TsvSerializer.class.getName().equals(outputFormatClass)) {
+ String delimiter = properties.getProperty("output.tsv.delimiter", "\t");
+ serializer = new TsvSerializer(fieldNames, delimiter.charAt(0));
+ } else if (CsvSerializer.class.getName().equals(outputFormatClass)) {
+ serializer = new CsvSerializer(fieldNames);
+ } else if (AvroSerializer.class.getName().equals(outputFormatClass)) {
+ String schemaString = properties.getProperty("output.avro.schema");
+ Preconditions.checkArgument(isNotEmpty(schemaString), "output.avro.schema can not be empty");
+ serializer = new AvroSerializer(schemaString, fieldNames);
+ } else {
+ serializer = Utils.newInstance(outputFormatClass);
+ }
+ } else {
+ //use JsonSerializer as the default serializer
+ serializer = new JsonSerializer(fieldNames);
+ }
+ return serializer;
+ }
+
+ public static Object convertAvroUtf8(Object value){
+ Object ret;
+ if (value instanceof Utf8) {
+ ret = value.toString();
+ } else if (value instanceof Map<?, ?>) {
+ ret = convertAvroUtf8Map((Map<Object,Object>)value);
+ } else if (value instanceof GenericData.Array) {
+ ret = convertAvroUtf8Array((GenericData.Array)value);
+ } else {
+ ret = value;
+ }
+ return ret;
+ }
+
+ public static Object convertAvroUtf8Map(Map<Object,Object> value) {
+ Map<Object, Object> map = new HashMap<>(value.size());
+ for (Map.Entry<Object, Object> entry : value.entrySet()) {
+ Object k = convertAvroUtf8(entry.getKey());
+ Object v = convertAvroUtf8(entry.getValue());
+ map.put(k, v);
+ }
+ return map;
+ }
+
+ public static Object convertAvroUtf8Array(GenericData.Array value){
+ List<Object> ls = new ArrayList<>(value.size());
+ for(Object o : value){
+ ls.add(convertAvroUtf8(o));
+ }
+ return ls;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
new file mode 100644
index 0000000..a0f3af3
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.utils;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public final class Utils {
+
+ /**
+ * This method for splitting string into parts by a delimiter
+ * It has higher performance than String.split(String regex)
+ *
+ * @param data
+ * @param delimiter
+ * @return
+ */
+ public static List<String> split(String data, char delimiter){
+ List<String> list = new LinkedList<>();
+ //do not use .toCharArray avoid system copy
+ StringBuilder sb = new StringBuilder(512);
+ int len = data.length();
+ for (int i=0; i < len; i++) {
+ char ch = data.charAt(i);
+ if(ch == delimiter){
+ list.add(sb.toString());
+ sb.setLength(0);
+ if(i == len - 1){
+ list.add("");
+ }
+ }else{
+ sb.append(ch);
+ }
+ }
+ if (sb.length() > 0) {
+ list.add(sb.toString());
+ }
+ return list;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..9a945f7
--- /dev/null
+++ b/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.storm.sql.runtime.datasource.socket.SocketDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
new file mode 100644
index 0000000..174bfde
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
+import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestAvroSerializer {
+ private static final String schemaString = "{\"type\":\"record\"," +
+ "\"name\":\"avrotest\"," +
+ "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
+ "{ \"name\":\"val\", \"type\":\"string\" }]}";
+
+ private static final String schemaString1 = "{\"type\":\"record\"," +
+ "\"name\":\"avrotest\"," +
+ "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
+ "{ \"type\":{\"type\":\"map\",\"values\": \"long\"}, \"name\":\"val1\" }," +
+ "{ \"type\":{\"type\":\"array\",\"items\": \"string\"}, \"name\":\"val2\" }]}";
+
+ @Test
+ public void testAvroSchemeAndSerializer() {
+ List<String> fields = Lists.newArrayList("ID", "val");
+ List<Object> o = Lists.newArrayList(1, "2");
+
+ AvroSerializer serializer = new AvroSerializer(schemaString, fields);
+ ByteBuffer byteBuffer = serializer.write(o, null);
+
+ AvroScheme scheme = new AvroScheme(schemaString, fields);
+ assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+ }
+
+ @Test
+ public void testAvroComplexSchemeAndSerializer() {
+ List<String> fields = Lists.newArrayList("ID", "val1", "val2");
+
+ Map<String,Long> mp = Maps.newHashMap();;
+ mp.put("l1",1234L);
+ mp.put("l2",56789L);
+ List<String> ls = Lists.newArrayList("s1", "s2");
+ List<Object> o = Lists.newArrayList(1, mp, ls);
+
+ AvroSerializer serializer = new AvroSerializer(schemaString1, fields);
+ ByteBuffer byteBuffer = serializer.write(o, null);
+
+ AvroScheme scheme = new AvroScheme(schemaString1, fields);
+ assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
new file mode 100644
index 0000000..0108949
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
+import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestCsvSerializer {
+
+ @Test
+ public void testCsvSchemeAndSerializer() {
+ List<String> fields = Lists.newArrayList("ID", "val");
+ List<Object> o = Lists.newArrayList("1", "2");
+
+ CsvSerializer serializer = new CsvSerializer(fields);
+ ByteBuffer byteBuffer = serializer.write(o, null);
+
+ CsvScheme scheme = new CsvScheme(fields);
+ assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+
+ // Fields with embedded commas or double-quote characters
+ fields = Lists.newArrayList("ID", "val", "v");
+ o = Lists.newArrayList("1,9", "2,\"3\",5", "\"8\"");
+
+ serializer = new CsvSerializer(fields);
+ byteBuffer = serializer.write(o, null);
+
+ scheme = new CsvScheme(fields);
+ assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
new file mode 100644
index 0000000..6ca1877
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import org.apache.storm.sql.runtime.serde.json.JsonScheme;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.utils.Utils;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestJsonRepresentation {
+ @Test
+ public void testJsonScheme() {
+ final List<String> fields = Lists.newArrayList("ID", "val");
+ final String s = "{\"ID\": 1, \"val\": \"2\"}";
+ JsonScheme scheme = new JsonScheme(fields);
+ List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
+ assertArrayEquals(new Object[] {1, "2"}, o.toArray());
+ }
+
+ @Test
+ public void testJsonSerializer() {
+ final List<String> fields = Lists.newArrayList("ID", "val");
+ List<Object> o = Lists.<Object> newArrayList(1, "2");
+ JsonSerializer s = new JsonSerializer(fields);
+ ByteBuffer buf = s.write(o, null);
+ byte[] b = Utils.toByteArray(buf);
+ assertEquals("{\"ID\":1,\"val\":\"2\"}", new String(b));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
new file mode 100644
index 0000000..1798828
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
+import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestTsvSerializer {
+
+ @Test
+ public void testTsvSchemeAndSerializer() {
+ final char delimiter = '\t';
+
+ List<String> fields = Lists.newArrayList("ID", "val");
+ List<Object> o = Lists.newArrayList("1", "2");
+
+ TsvSerializer serializer = new TsvSerializer(fields, delimiter);
+ ByteBuffer byteBuffer = serializer.write(o, null);
+
+ TsvScheme scheme = new TsvScheme(fields, delimiter);
+ assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
new file mode 100644
index 0000000..0a3bac6
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
@@ -0,0 +1,584 @@
+/*
+ * *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements. See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership. The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License. You may obtain a copy of the License at
+ * * <p>
+ * * http://www.apache.org/licenses/LICENSE-2.0
+ * * <p>
+ * * Unless required by applicable law or agreed to in writing, software
+ * * distributed under the License is distributed on an "AS IS" BASIS,
+ * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * * See the License for the specific language governing permissions and
+ * * limitations under the License.
+ *
+ */
+package org.apache.storm.sql;
+
+import org.apache.storm.sql.runtime.ChannelContext;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.task.IMetricsContext;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.spout.IBatchSpout;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public class TestUtils {
+ public static class MyPlus {
+ public static Integer evaluate(Integer x, Integer y) {
+ return x + y;
+ }
+ }
+
+ public static class MyConcat {
+ public static String init() {
+ return "";
+ }
+ public static String add(String accumulator, String val) {
+ return accumulator + val;
+ }
+ public static String result(String accumulator) {
+ return accumulator;
+ }
+ }
+
+
+ public static class TopN {
+ public static PriorityQueue<Integer> init() {
+ return new PriorityQueue<>();
+ }
+ public static PriorityQueue<Integer> add(PriorityQueue<Integer> accumulator, Integer n, Integer val) {
+ if (n <= 0) {
+ return accumulator;
+ }
+ if (accumulator.size() >= n) {
+ if (val > accumulator.peek()) {
+ accumulator.remove();
+ accumulator.add(val);
+ }
+ } else {
+ accumulator.add(val);
+ }
+ return accumulator;
+ }
+ public static List<Integer> result(PriorityQueue<Integer> accumulator) {
+ List<Integer> res = new ArrayList<>(accumulator);
+ Collections.reverse(res);
+ return res;
+ }
+ }
+
+
+ public static class MockDataSource implements DataSource {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+ public MockDataSource() {
+ for (int i = 0; i < 5; ++i) {
+ RECORDS.add(new Values(i, "x", null));
+ }
+ }
+
+ @Override
+ public void open(ChannelContext ctx) {
+ for (Values v : RECORDS) {
+ ctx.emit(v);
+ }
+ ctx.fireChannelInactive();
+ }
+ }
+
+ public static class MockGroupDataSource implements DataSource {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+ public MockGroupDataSource() {
+ for (int i = 0; i < 10; ++i) {
+ RECORDS.add(new Values(i/3, i, (i+1)* 0.5, "x", i/2));
+ }
+ }
+
+ @Override
+ public void open(ChannelContext ctx) {
+ for (Values v : RECORDS) {
+ ctx.emit(v);
+ }
+ // force evaluation of the aggregate function on the last group
+ ctx.flush();
+ ctx.fireChannelInactive();
+ }
+ }
+
+ public static class MockEmpDataSource implements DataSource {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+ public MockEmpDataSource() {
+ RECORDS.add(new Values(1, "emp1", 1));
+ RECORDS.add(new Values(2, "emp2", 1));
+ RECORDS.add(new Values(3, "emp3", 2));
+ }
+
+ @Override
+ public void open(ChannelContext ctx) {
+ for (Values v : RECORDS) {
+ ctx.emit(v);
+ }
+ ctx.flush();
+ ctx.fireChannelInactive();
+ }
+ }
+
+ public static class MockDeptDataSource implements DataSource {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+ public MockDeptDataSource() {
+ RECORDS.add(new Values(1, "dept1"));
+ RECORDS.add(new Values(2, "dept2"));
+ RECORDS.add(new Values(3, "dept3"));
+ }
+
+ @Override
+ public void open(ChannelContext ctx) {
+ for (Values v : RECORDS) {
+ ctx.emit(v);
+ }
+ ctx.flush();
+ ctx.fireChannelInactive();
+ }
+ }
+
+ public static class MockNestedDataSource implements DataSource {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+ public MockNestedDataSource() {
+ List<Integer> ints = Arrays.asList(100, 200, 300);
+ for (int i = 0; i < 5; ++i) {
+ Map<String, Integer> map = new HashMap<>();
+ map.put("b", i);
+ map.put("c", i*i);
+ Map<String, Map<String, Integer>> mm = new HashMap<>();
+ mm.put("a", map);
+ RECORDS.add(new Values(i, map, mm, ints));
+ }
+ }
+
+ @Override
+ public void open(ChannelContext ctx) {
+ for (Values v : RECORDS) {
+ ctx.emit(v);
+ }
+ ctx.fireChannelInactive();
+ }
+ }
+
+ public static class MockState implements State {
+ /**
+ * Collect all values in a static variable as the instance will go through serialization and deserialization.
+ * NOTE: This should be cleared before or after running each test.
+ */
+ private transient static final List<List<Object> > VALUES = new ArrayList<>();
+
+ public static List<List<Object>> getCollectedValues() {
+ return VALUES;
+ }
+
+ @Override
+ public void beginCommit(Long txid) {
+ // NOOP
+ }
+
+ @Override
+ public void commit(Long txid) {
+ // NOOP
+ }
+
+ public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
+ for (TridentTuple tuple : tuples) {
+ VALUES.add(tuple.getValues());
+ }
+ }
+ }
+
+ public static class MockStateFactory implements StateFactory {
+
+ @Override
+ public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+ return new MockState();
+ }
+ }
+
+ public static class MockStateUpdater implements StateUpdater<MockState> {
+
+ @Override
+ public void updateState(MockState state, List<TridentTuple> tuples, TridentCollector collector) {
+ state.updateState(tuples, collector);
+ }
+
+ @Override
+ public void prepare(Map conf, TridentOperationContext context) {
+ // NOOP
+ }
+
+ @Override
+ public void cleanup() {
+ // NOOP
+ }
+ }
+
+ public static class MockSqlTridentDataSource implements ISqlTridentDataSource {
+ @Override
+ public IBatchSpout getProducer() {
+ return new MockSpout();
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+ }
+
+ private static class MockSpout implements IBatchSpout {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+ private final Fields OUTPUT_FIELDS = new Fields("ID", "NAME", "ADDR");
+
+ public MockSpout() {
+ RECORDS.add(new Values(0, "a", "y"));
+ RECORDS.add(new Values(1, "ab", "y"));
+ RECORDS.add(new Values(2, "abc", "y"));
+ RECORDS.add(new Values(3, "abcd", "y"));
+ RECORDS.add(new Values(4, "abcde", "y"));
+ }
+
+ private boolean emitted = false;
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ if (emitted) {
+ return;
+ }
+
+ for (Values r : RECORDS) {
+ collector.emit(r);
+ }
+ emitted = true;
+ }
+
+ @Override
+ public void ack(long batchId) {
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return OUTPUT_FIELDS;
+ }
+ }
+ }
+
+ public static class MockSqlTridentGroupedDataSource implements ISqlTridentDataSource {
+ @Override
+ public IBatchSpout getProducer() {
+ return new MockGroupedSpout();
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+ }
+
+ private static class MockGroupedSpout implements IBatchSpout {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+ private final Fields OUTPUT_FIELDS = new Fields("ID", "GRPID", "NAME", "ADDR", "AGE", "SCORE");
+
+ public MockGroupedSpout() {
+ for (int i = 0; i < 5; ++i) {
+ RECORDS.add(new Values(i, 0, "x", "y", 5 - i, i * 10));
+ }
+ }
+
+ private boolean emitted = false;
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ if (emitted) {
+ return;
+ }
+
+ for (Values r : RECORDS) {
+ collector.emit(r);
+ }
+ emitted = true;
+ }
+
+ @Override
+ public void ack(long batchId) {
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return OUTPUT_FIELDS;
+ }
+ }
+ }
+
+ public static class MockSqlTridentJoinDataSourceEmp implements ISqlTridentDataSource {
+ @Override
+ public IBatchSpout getProducer() {
+ return new MockSpout();
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+ }
+
+ private static class MockSpout implements IBatchSpout {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+ private final Fields OUTPUT_FIELDS = new Fields("EMPID", "EMPNAME", "DEPTID");
+
+ public MockSpout() {
+ for (int i = 0; i < 5; ++i) {
+ RECORDS.add(new Values(i, "emp-" + i, i % 2));
+ }
+ for (int i = 10; i < 15; ++i) {
+ RECORDS.add(new Values(i, "emp-" + i, i));
+ }
+ }
+
+ private boolean emitted = false;
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ if (emitted) {
+ return;
+ }
+
+ for (Values r : RECORDS) {
+ collector.emit(r);
+ }
+ emitted = true;
+ }
+
+ @Override
+ public void ack(long batchId) {
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return OUTPUT_FIELDS;
+ }
+ }
+ }
+
+ public static class MockSqlTridentJoinDataSourceDept implements ISqlTridentDataSource {
+ @Override
+ public IBatchSpout getProducer() {
+ return new MockSpout();
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+ }
+
+ private static class MockSpout implements IBatchSpout {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+ private final Fields OUTPUT_FIELDS = new Fields("DEPTID", "DEPTNAME");
+
+ public MockSpout() {
+ for (int i = 0; i < 5; ++i) {
+ RECORDS.add(new Values(i, "dept-" + i));
+ }
+ }
+
+ private boolean emitted = false;
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ if (emitted) {
+ return;
+ }
+
+ for (Values r : RECORDS) {
+ collector.emit(r);
+ }
+ emitted = true;
+ }
+
+ @Override
+ public void ack(long batchId) {
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return OUTPUT_FIELDS;
+ }
+ }
+ }
+
+ public static class MockSqlTridentNestedDataSource implements ISqlTridentDataSource {
+ @Override
+ public IBatchSpout getProducer() {
+ return new MockSpout();
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+ }
+
+ private static class MockSpout implements IBatchSpout {
+ private final ArrayList<Values> RECORDS = new ArrayList<>();
+ private final Fields OUTPUT_FIELDS = new Fields("ID", "MAPFIELD", "NESTEDMAPFIELD", "ARRAYFIELD");
+
+ public MockSpout() {
+ List<Integer> ints = Arrays.asList(100, 200, 300);
+ for (int i = 0; i < 5; ++i) {
+ Map<String, Integer> map = new HashMap<>();
+ map.put("b", i);
+ map.put("c", i*i);
+ Map<String, Map<String, Integer>> mm = new HashMap<>();
+ mm.put("a", map);
+ RECORDS.add(new Values(i, map, mm, ints));
+ }
+ }
+
+ private boolean emitted = false;
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ if (emitted) {
+ return;
+ }
+
+ for (Values r : RECORDS) {
+ collector.emit(r);
+ }
+ emitted = true;
+ }
+
+ @Override
+ public void ack(long batchId) {
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return OUTPUT_FIELDS;
+ }
+ }
+ }
+
+ public static class CollectDataChannelHandler implements ChannelHandler {
+ private final List<Values> values;
+
+ public CollectDataChannelHandler(List<Values> values) {
+ this.values = values;
+ }
+
+ @Override
+ public void dataReceived(ChannelContext ctx, Values data) {
+ values.add(data);
+ }
+
+ @Override
+ public void channelInactive(ChannelContext ctx) {}
+
+ @Override
+ public void exceptionCaught(Throwable cause) {
+ throw new RuntimeException(cause);
+ }
+
+ @Override
+ public void flush(ChannelContext ctx) {}
+
+ @Override
+ public void setSource(ChannelContext ctx, Object source) {}
+ }
+
+ public static long monotonicNow() {
+ final long NANOSECONDS_PER_MILLISECOND = 1000000;
+ return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
new file mode 100644
index 0000000..1e8edee
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.datasource.socket;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestSocketDataSourceProvider {
+ private static final List<FieldInfo> FIELDS = ImmutableList.of(
+ new FieldInfo("ID", int.class, true),
+ new FieldInfo("val", String.class, false));
+ private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+ private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+
+ @Test
+ public void testSocketSink() throws IOException {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create("socket://localhost:8888"), null, null, new Properties(), FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(SocketState.Factory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(SocketStateUpdater.class, consumer.getStateUpdater().getClass());
+
+ // makeState() fails on creating State so we just mock SocketState anyway
+ SocketState mockState = mock(SocketState.class);
+ StateUpdater stateUpdater = consumer.getStateUpdater();
+
+ List<TridentTuple> tupleList = mockTupleList();
+
+ stateUpdater.updateState(mockState, tupleList, null);
+ for (TridentTuple t : tupleList) {
+ String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+ verify(mockState).write(serializedValue + "\n");
+ }
+ }
+
+ private static List<TridentTuple> mockTupleList() {
+ List<TridentTuple> tupleList = new ArrayList<>();
+ TridentTuple t0 = mock(TridentTuple.class);
+ TridentTuple t1 = mock(TridentTuple.class);
+ when(t0.getValueByField("ID")).thenReturn(1);
+ when(t0.getValueByField("val")).thenReturn("2");
+ doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+ when(t0.size()).thenReturn(2);
+
+ when(t1.getValueByField("ID")).thenReturn(2);
+ when(t1.getValueByField("val")).thenReturn("3");
+ doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+ when(t1.size()).thenReturn(2);
+
+ tupleList.add(t0);
+ tupleList.add(t1);
+ return tupleList;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-dist/binary/final-package/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/final-package/src/main/assembly/binary.xml b/storm-dist/binary/final-package/src/main/assembly/binary.xml
index a6aab07..7ee1445 100644
--- a/storm-dist/binary/final-package/src/main/assembly/binary.xml
+++ b/storm-dist/binary/final-package/src/main/assembly/binary.xml
@@ -298,57 +298,6 @@
</includes>
</fileSet>
<fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-core/target/app-assembler/repo</directory>
- <outputDirectory>external/sql/storm-sql-core</outputDirectory>
- <includes>
- <include>*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-kafka/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-kafka</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-redis/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-redis</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-mongodb/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-mongodb</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-hdfs/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-hdfs</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-runtime/target/app-assembler/repo</directory>
- <outputDirectory>external/sql/storm-sql-runtime</outputDirectory>
- <includes>
- <include>*jar</include>
- </includes>
- </fileSet>
-
- <fileSet>
- <directory>${project.basedir}/../../../external/sql</directory>
- <outputDirectory>external/sql</outputDirectory>
- <includes>
- <include>README.*</include>
- </includes>
- </fileSet>
-
- <fileSet>
<directory>${project.basedir}/../../../external/storm-mqtt</directory>
<outputDirectory>external/storm-mqtt</outputDirectory>
<includes>
@@ -426,13 +375,6 @@
</includes>
</fileSet>
<fileSet>
- <directory>${project.basedir}/../../../external/storm-submit-tools/target</directory>
- <outputDirectory>external/storm-submit-tools</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
<directory>${project.basedir}/../../../external/storm-jms</directory>
<outputDirectory>external/storm-jms</outputDirectory>
<includes>
@@ -504,7 +446,7 @@
<!-- $STORM_HOME/toollib -->
<fileSet>
<directory>${project.basedir}/../../../external/storm-kafka-monitor/target</directory>
- <outputDirectory>toollib</outputDirectory>
+ <outputDirectory>lib-tools/storm-kafka-monitor</outputDirectory>
<includes>
<include>storm*jar</include>
</includes>
@@ -539,6 +481,32 @@
</includes>
</fileSet>
+ <!-- storm-sql -->
+ <fileSet>
+ <directory>${project.basedir}/../../../sql/storm-sql-core/target/app-assembler/repo</directory>
+ <outputDirectory>lib-tools/sql/core</outputDirectory>
+ <includes>
+ <include>*jar</include>
+ </includes>
+ </fileSet>
+ <fileSet>
+ <directory>${project.basedir}/../../../sql/storm-sql-runtime/target/app-assembler/repo</directory>
+ <outputDirectory>lib-tools/sql/runtime</outputDirectory>
+ <includes>
+ <include>*jar</include>
+ </includes>
+ </fileSet>
+
+ <!-- storm-submit-tools -->
+ <fileSet>
+ <directory>${project.basedir}/../../../storm-submit-tools/target</directory>
+ <outputDirectory>lib-tools/submit-tools</outputDirectory>
+ <includes>
+ <include>storm*jar</include>
+ </includes>
+ </fileSet>
+
+
</fileSets>
<files>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/pom.xml
----------------------------------------------------------------------
diff --git a/storm-submit-tools/pom.xml b/storm-submit-tools/pom.xml
new file mode 100644
index 0000000..c58930b
--- /dev/null
+++ b/storm-submit-tools/pom.xml
@@ -0,0 +1,216 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+ http://www.apache.org/licenses/LICENSE-2.0
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>storm-submit-tools</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.googlecode.json-simple</groupId>
+ <artifactId>json-simple</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+
+ <!-- Aether :: maven dependency resolution -->
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-plugin-api</artifactId>
+ <version>3.0</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.codehaus.plexus</groupId>
+ <artifactId>plexus-utils</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.sonatype.sisu</groupId>
+ <artifactId>sisu-inject-plexus</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-model</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-api</artifactId>
+ <version>1.12</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-util</artifactId>
+ <version>1.12</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-impl</artifactId>
+ <version>1.12</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-aether-provider</artifactId>
+ <version>3.0.3</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-spi</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-util</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-impl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.plexus</groupId>
+ <artifactId>plexus-utils</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-connector-file</artifactId>
+ <version>1.12</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.sonatype.aether</groupId>
+ <artifactId>aether-connector-wagon</artifactId>
+ <version>1.12</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.maven.wagon</groupId>
+ <artifactId>wagon-provider-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.maven.wagon</groupId>
+ <artifactId>wagon-provider-api</artifactId>
+ <version>1.0</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.codehaus.plexus</groupId>
+ <artifactId>plexus-utils</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.maven.wagon</groupId>
+ <artifactId>wagon-http-lightweight</artifactId>
+ <version>1.0</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.maven.wagon</groupId>
+ <artifactId>wagon-http-shared</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.maven.wagon</groupId>
+ <artifactId>wagon-http</artifactId>
+ <version>1.0</version>
+ <exclusions>
+ </exclusions>
+ </dependency>
+
+ <!-- storm-core is needed only for test (surefire) -->
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <configuration>
+ <keepDependenciesWithProvidedScope>false</keepDependenciesWithProvidedScope>
+ <createDependencyReducedPom>true</createDependencyReducedPom>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.sf</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.dsa</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ <exclude>META-INF/*.rsa</exclude>
+ <exclude>META-INF/*.EC</exclude>
+ <exclude>META-INF/*.ec</exclude>
+ <exclude>META-INF/MSFTSIG.SF</exclude>
+ <exclude>META-INF/MSFTSIG.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <transformers>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+</project>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
new file mode 100644
index 0000000..ac50110
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.submit.command;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.submit.dependency.AetherUtils;
+import org.apache.storm.submit.dependency.DependencyResolver;
+import org.json.simple.JSONValue;
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.resolution.ArtifactResult;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DependencyResolverMain {
+
+ public static void main(String[] args) {
+ if (args.length < 1) {
+ throw new IllegalArgumentException("artifacts must be presented.");
+ }
+
+ String artifactsArg = args[0];
+
+ // DO NOT CHANGE THIS TO SYSOUT
+ System.err.println("DependencyResolver input - artifacts: " + artifactsArg);
+ List<Dependency> dependencies = parseArtifactArgs(artifactsArg);
+
+ List<RemoteRepository> repositories;
+ if (args.length > 1) {
+ String remoteRepositoryArg = args[1];
+
+ // DO NOT CHANGE THIS TO SYSOUT
+ System.err.println("DependencyResolver input - repositories: " + remoteRepositoryArg);
+
+ repositories = parseRemoteRepositoryArgs(remoteRepositoryArg);
+ } else {
+ repositories = Collections.emptyList();
+ }
+
+ try {
+ String localMavenRepoPath = getOrDefaultLocalMavenRepositoryPath("local-repo");
+ DependencyResolver resolver = new DependencyResolver(localMavenRepoPath, repositories);
+
+ List<ArtifactResult> artifactResults = resolver.resolve(dependencies);
+
+ Iterable<ArtifactResult> missingArtifacts = filterMissingArtifacts(artifactResults);
+ if (missingArtifacts.iterator().hasNext()) {
+ printMissingArtifactsToSysErr(missingArtifacts);
+ throw new RuntimeException("Some artifacts are not resolved");
+ }
+
+ System.out.println(JSONValue.toJSONString(transformArtifactResultToArtifactToPaths(artifactResults)));
+ System.out.flush();
+ } catch (Throwable e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static Iterable<ArtifactResult> filterMissingArtifacts(List<ArtifactResult> artifactResults) {
+ return Iterables.filter(artifactResults, new Predicate<ArtifactResult>() {
+ @Override
+ public boolean apply(ArtifactResult artifactResult) {
+ return artifactResult.isMissing();
+ }
+ });
+ }
+
+ private static void printMissingArtifactsToSysErr(Iterable<ArtifactResult> missingArtifacts) {
+ for (ArtifactResult artifactResult : missingArtifacts) {
+ System.err.println("ArtifactResult : " + artifactResult + " / Errors : " + artifactResult.getExceptions());
+ }
+ }
+
+ private static List<Dependency> parseArtifactArgs(String artifactArgs) {
+ List<String> artifacts = Arrays.asList(artifactArgs.split(","));
+ List<Dependency> dependencies = new ArrayList<>(artifacts.size());
+ for (String artifactOpt : artifacts) {
+ if (artifactOpt.trim().isEmpty()) {
+ continue;
+ }
+
+ dependencies.add(AetherUtils.parseDependency(artifactOpt));
+ }
+
+ return dependencies;
+ }
+
+ private static List<RemoteRepository> parseRemoteRepositoryArgs(String remoteRepositoryArg) {
+ List<String> repositories = Arrays.asList(remoteRepositoryArg.split(","));
+ List<RemoteRepository> remoteRepositories = new ArrayList<>(repositories.size());
+ for (String repositoryOpt : repositories) {
+ if (repositoryOpt.trim().isEmpty()) {
+ continue;
+ }
+
+ remoteRepositories.add(AetherUtils.parseRemoteRepository(repositoryOpt));
+ }
+
+ return remoteRepositories;
+ }
+
+ private static Map<String, String> transformArtifactResultToArtifactToPaths(List<ArtifactResult> artifactResults) {
+ Map<String, String> artifactToPath = new LinkedHashMap<>();
+ for (ArtifactResult artifactResult : artifactResults) {
+ Artifact artifact = artifactResult.getArtifact();
+ artifactToPath.put(AetherUtils.artifactToString(artifact), artifact.getFile().getAbsolutePath());
+ }
+ return artifactToPath;
+ }
+
+ private static String getOrDefaultLocalMavenRepositoryPath(String defaultPath) {
+ String localMavenRepoPathStr = getLocalMavenRepositoryPath();
+ if (StringUtils.isNotEmpty(localMavenRepoPathStr)) {
+ Path localMavenRepoPath = new File(localMavenRepoPathStr).toPath();
+ if (Files.exists(localMavenRepoPath) && Files.isDirectory(localMavenRepoPath)) {
+ return localMavenRepoPathStr;
+ }
+ }
+
+ return defaultPath;
+ }
+
+ private static String getLocalMavenRepositoryPath() {
+ String userHome = System.getProperty("user.home");
+ if (StringUtils.isNotEmpty(userHome)) {
+ return userHome + File.separator + ".m2" + File.separator + "repository";
+ }
+
+ return null;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
new file mode 100644
index 0000000..086be2b
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.submit.dependency;
+
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.Exclusion;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AetherUtils {
+ private AetherUtils() {
+ }
+
+ public static Dependency parseDependency(String dependency) {
+ List<String> dependencyAndExclusions = Arrays.asList(dependency.split("\\^"));
+ Collection<Exclusion> exclusions = new ArrayList<>();
+ for (int idx = 1 ; idx < dependencyAndExclusions.size() ; idx++) {
+ exclusions.add(AetherUtils.createExclusion(dependencyAndExclusions.get(idx)));
+ }
+
+ Artifact artifact = new DefaultArtifact(dependencyAndExclusions.get(0));
+ return new Dependency(artifact, JavaScopes.COMPILE, false, exclusions);
+ }
+
+ public static Exclusion createExclusion(String exclusionString) {
+ String[] parts = exclusionString.split(":");
+
+ // length of parts should be greater than 0
+ String groupId = parts[0];
+
+ String artifactId = "*";
+ String classifier = "*";
+ String extension = "*";
+
+ int len = parts.length;
+ if (len > 1) {
+ artifactId = parts[1];
+ }
+ if (len > 2) {
+ classifier = parts[2];
+ }
+ if (len > 3) {
+ extension = parts[3];
+ }
+
+ return new Exclusion(groupId, artifactId, classifier, extension);
+ }
+
+ public static String artifactToString(Artifact artifact) {
+ StringBuilder buffer = new StringBuilder(128);
+ buffer.append(artifact.getGroupId());
+ buffer.append(':').append(artifact.getArtifactId());
+ buffer.append(':').append(artifact.getExtension());
+ if (artifact.getClassifier().length() > 0) {
+ buffer.append(':').append(artifact.getClassifier());
+ }
+ buffer.append(':').append(artifact.getVersion());
+ return buffer.toString();
+ }
+
+ public static RemoteRepository parseRemoteRepository(String repository) {
+ String[] parts = repository.split("\\^");
+ if (parts.length < 2) {
+ throw new IllegalArgumentException("Bad remote repository form: " + repository);
+ }
+
+ return new RemoteRepository(parts[0], "default", parts[1]);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
new file mode 100644
index 0000000..816e3cc
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.submit.dependency;
+
+import org.apache.maven.repository.internal.MavenRepositorySystemSession;
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.RepositorySystemSession;
+import org.sonatype.aether.repository.LocalRepository;
+import org.sonatype.aether.repository.RemoteRepository;
+
+import java.io.File;
+
+/**
+ * Manage mvn repository.
+ */
+public class Booter {
+ public static RepositorySystem newRepositorySystem() {
+ return RepositorySystemFactory.newRepositorySystem();
+ }
+
+ public static RepositorySystemSession newRepositorySystemSession(
+ RepositorySystem system, String localRepoPath) {
+ MavenRepositorySystemSession session = new MavenRepositorySystemSession();
+
+ LocalRepository localRepo =
+ new LocalRepository(new File(localRepoPath).getAbsolutePath());
+ session.setLocalRepositoryManager(system.newLocalRepositoryManager(localRepo));
+
+ return session;
+ }
+
+ public static RemoteRepository newCentralRepository() {
+ return new RemoteRepository("central", "default", "http://repo1.maven.org/maven2/");
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
new file mode 100644
index 0000000..4534344
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.submit.dependency;
+
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.RepositorySystemSession;
+import org.sonatype.aether.collection.CollectRequest;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.DependencyFilter;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.resolution.ArtifactResolutionException;
+import org.sonatype.aether.resolution.ArtifactResult;
+import org.sonatype.aether.resolution.DependencyRequest;
+import org.sonatype.aether.resolution.DependencyResolutionException;
+import org.sonatype.aether.util.artifact.JavaScopes;
+import org.sonatype.aether.util.filter.DependencyFilterUtils;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class DependencyResolver {
+ private final RepositorySystem system = Booter.newRepositorySystem();
+ private final RepositorySystemSession session;
+
+ private final List<RemoteRepository> remoteRepositories;
+
+ public DependencyResolver(String localRepoPath) {
+ this(localRepoPath, Collections.emptyList());
+ }
+
+ public DependencyResolver(String localRepoPath, List<RemoteRepository> repositories) {
+ localRepoPath = handleRelativePath(localRepoPath);
+
+ session = Booter.newRepositorySystemSession(system, localRepoPath);
+
+ remoteRepositories = new ArrayList<>();
+ remoteRepositories.add(Booter.newCentralRepository());
+ remoteRepositories.addAll(repositories);
+ }
+
+ private String handleRelativePath(String localRepoPath) {
+ File repoDir = new File(localRepoPath);
+ if (!repoDir.isAbsolute()) {
+ // find homedir
+ String home = System.getProperty("storm.home");
+ if (home == null) {
+ home = ".";
+ }
+
+ localRepoPath = home + "/" + localRepoPath;
+ }
+ return localRepoPath;
+ }
+
+ public List<ArtifactResult> resolve(List<Dependency> dependencies) throws MalformedURLException,
+ DependencyResolutionException, ArtifactResolutionException {
+
+ DependencyFilter classpathFilter = DependencyFilterUtils
+ .classpathFilter(JavaScopes.COMPILE, JavaScopes.RUNTIME);
+
+ if (dependencies.size() == 0) {
+ return Collections.EMPTY_LIST;
+ }
+
+ CollectRequest collectRequest = new CollectRequest();
+ collectRequest.setRoot(dependencies.get(0));
+ for (int idx = 1; idx < dependencies.size(); idx++) {
+ collectRequest.addDependency(dependencies.get(idx));
+ }
+
+ for (RemoteRepository repository : remoteRepositories) {
+ collectRequest.addRepository(repository);
+ }
+
+ DependencyRequest dependencyRequest = new DependencyRequest(collectRequest, classpathFilter);
+ return system.resolveDependencies(session, dependencyRequest).getArtifactResults();
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
new file mode 100644
index 0000000..f1fa2db
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.submit.dependency;
+
+import org.apache.maven.repository.internal.DefaultServiceLocator;
+import org.apache.maven.wagon.Wagon;
+import org.apache.maven.wagon.providers.http.HttpWagon;
+import org.apache.maven.wagon.providers.http.LightweightHttpWagon;
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.connector.file.FileRepositoryConnectorFactory;
+import org.sonatype.aether.connector.wagon.WagonProvider;
+import org.sonatype.aether.connector.wagon.WagonRepositoryConnectorFactory;
+import org.sonatype.aether.spi.connector.RepositoryConnectorFactory;
+
+/**
+ * Get maven repository instance.
+ */
+public class RepositorySystemFactory {
+ public static RepositorySystem newRepositorySystem() {
+ DefaultServiceLocator locator = new DefaultServiceLocator();
+ locator.addService(RepositoryConnectorFactory.class, FileRepositoryConnectorFactory.class);
+ locator.addService(RepositoryConnectorFactory.class, WagonRepositoryConnectorFactory.class);
+ locator.setServices(WagonProvider.class, new ManualWagonProvider());
+
+ return locator.getService(RepositorySystem.class);
+ }
+
+ /**
+ * ManualWagonProvider
+ */
+ public static class ManualWagonProvider implements WagonProvider {
+
+ @Override
+ public Wagon lookup(String roleHint) throws Exception {
+ if ("http".equals(roleHint)) {
+ return new LightweightHttpWagon();
+ }
+
+ if ("https".equals(roleHint)) {
+ return new HttpWagon();
+ }
+
+ return null;
+ }
+
+ @Override
+ public void release(Wagon arg0) {
+
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
new file mode 100644
index 0000000..9951621
--- /dev/null
+++ b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.submit.dependency;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.Exclusion;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class AetherUtilsTest {
+ @Test
+ public void parseDependency() throws Exception {
+ String testDependency = "testgroup:testartifact:1.0.0^testgroup:testexcartifact^testgroup:*";
+
+ Dependency dependency = AetherUtils.parseDependency(testDependency);
+
+ assertEquals("testgroup", dependency.getArtifact().getGroupId());
+ assertEquals("testartifact", dependency.getArtifact().getArtifactId());
+ assertEquals("1.0.0", dependency.getArtifact().getVersion());
+ assertEquals(JavaScopes.COMPILE, dependency.getScope());
+
+ assertEquals(2, dependency.getExclusions().size());
+
+ List<Exclusion> exclusions = Lists.newArrayList(dependency.getExclusions());
+
+ Exclusion exclusion = exclusions.get(0);
+ assertEquals("testgroup", exclusion.getGroupId());
+ assertEquals("testexcartifact", exclusion.getArtifactId());
+ assertEquals(JavaScopes.COMPILE, dependency.getScope());
+
+ exclusion = exclusions.get(1);
+ assertEquals("testgroup", exclusion.getGroupId());
+ assertEquals("*", exclusion.getArtifactId());
+ assertEquals(JavaScopes.COMPILE, dependency.getScope());
+ }
+
+ @Test
+ public void createExclusion() throws Exception {
+ String testExclusion = "group";
+ Exclusion exclusion = AetherUtils.createExclusion(testExclusion);
+
+ assertEquals("group", exclusion.getGroupId());
+ assertEquals("*", exclusion.getArtifactId());
+ assertEquals("*", exclusion.getClassifier());
+ assertEquals("*", exclusion.getExtension());
+
+ testExclusion = "group:artifact";
+ exclusion = AetherUtils.createExclusion(testExclusion);
+
+ assertEquals("group", exclusion.getGroupId());
+ assertEquals("artifact", exclusion.getArtifactId());
+ assertEquals("*", exclusion.getClassifier());
+ assertEquals("*", exclusion.getExtension());
+
+ testExclusion = "group:artifact:site";
+ exclusion = AetherUtils.createExclusion(testExclusion);
+
+ assertEquals("group", exclusion.getGroupId());
+ assertEquals("artifact", exclusion.getArtifactId());
+ assertEquals("site", exclusion.getClassifier());
+ assertEquals("*", exclusion.getExtension());
+
+ testExclusion = "group:artifact:site:jar";
+ exclusion = AetherUtils.createExclusion(testExclusion);
+
+ assertEquals("group", exclusion.getGroupId());
+ assertEquals("artifact", exclusion.getArtifactId());
+ assertEquals("site", exclusion.getClassifier());
+ assertEquals("jar", exclusion.getExtension());
+ }
+
+ @Test
+ public void artifactToString() throws Exception {
+ Artifact testArtifact = new DefaultArtifact("org.apache.storm:storm-core:1.0.0");
+
+ String ret = AetherUtils.artifactToString(testArtifact);
+ assertEquals("org.apache.storm:storm-core:jar:1.0.0", ret);
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
new file mode 100644
index 0000000..086abad
--- /dev/null
+++ b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.submit.dependency;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.io.FileUtils;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.resolution.ArtifactResult;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+public class DependencyResolverTest {
+ private static Path tempDirForTest;
+
+ private DependencyResolver sut;
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ tempDirForTest = Files.createTempDirectory("dr-test");
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ FileUtils.deleteQuietly(tempDirForTest.toFile());
+ }
+
+ @Before
+ public void setUp() {
+ sut = new DependencyResolver(tempDirForTest.toAbsolutePath().toString());
+ }
+
+ @Test
+ public void resolveValid() throws Exception {
+ // please pick small artifact which has small transitive dependency
+ // and let's mark as Ignore if we want to run test even without internet or maven central is often not stable
+ Dependency dependency = new Dependency(new DefaultArtifact("org.apache.storm:flux-core:1.0.0"), JavaScopes.COMPILE);
+ List<ArtifactResult> results = sut.resolve(Lists.newArrayList(dependency));
+
+ assertTrue(results.size() > 0);
+ // it should be org.apache.storm:flux-core:jar:1.0.0 and commons-cli:commons-cli:jar:1.2
+ assertContains(results, "org.apache.storm", "flux-core", "1.0.0");
+ assertContains(results, "commons-cli", "commons-cli", "1.2");
+ }
+
+ private void assertContains(List<ArtifactResult> results, String groupId, String artifactId, String version) {
+ for (ArtifactResult result : results) {
+ if (result.getArtifact().getGroupId().equals(groupId) &&
+ result.getArtifact().getArtifactId().equals(artifactId) &&
+ result.getArtifact().getVersion().equals(version) &&
+ result.isResolved()) {
+ return;
+ }
+ }
+
+ throw new AssertionError("Result doesn't contain expected artifact > " + groupId + ":" + artifactId + ":" + version);
+ }
+
+}
\ No newline at end of file
[11/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
deleted file mode 100644
index 0a3bac6..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
+++ /dev/null
@@ -1,584 +0,0 @@
-/*
- * *
- * * Licensed to the Apache Software Foundation (ASF) under one
- * * or more contributor license agreements. See the NOTICE file
- * * distributed with this work for additional information
- * * regarding copyright ownership. The ASF licenses this file
- * * to you under the Apache License, Version 2.0 (the
- * * "License"); you may not use this file except in compliance
- * * with the License. You may obtain a copy of the License at
- * * <p>
- * * http://www.apache.org/licenses/LICENSE-2.0
- * * <p>
- * * Unless required by applicable law or agreed to in writing, software
- * * distributed under the License is distributed on an "AS IS" BASIS,
- * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * * See the License for the specific language governing permissions and
- * * limitations under the License.
- *
- */
-package org.apache.storm.sql;
-
-import org.apache.storm.sql.runtime.ChannelContext;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.spout.IBatchSpout;
-import org.apache.storm.trident.state.State;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-
-public class TestUtils {
- public static class MyPlus {
- public static Integer evaluate(Integer x, Integer y) {
- return x + y;
- }
- }
-
- public static class MyConcat {
- public static String init() {
- return "";
- }
- public static String add(String accumulator, String val) {
- return accumulator + val;
- }
- public static String result(String accumulator) {
- return accumulator;
- }
- }
-
-
- public static class TopN {
- public static PriorityQueue<Integer> init() {
- return new PriorityQueue<>();
- }
- public static PriorityQueue<Integer> add(PriorityQueue<Integer> accumulator, Integer n, Integer val) {
- if (n <= 0) {
- return accumulator;
- }
- if (accumulator.size() >= n) {
- if (val > accumulator.peek()) {
- accumulator.remove();
- accumulator.add(val);
- }
- } else {
- accumulator.add(val);
- }
- return accumulator;
- }
- public static List<Integer> result(PriorityQueue<Integer> accumulator) {
- List<Integer> res = new ArrayList<>(accumulator);
- Collections.reverse(res);
- return res;
- }
- }
-
-
- public static class MockDataSource implements DataSource {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
-
- public MockDataSource() {
- for (int i = 0; i < 5; ++i) {
- RECORDS.add(new Values(i, "x", null));
- }
- }
-
- @Override
- public void open(ChannelContext ctx) {
- for (Values v : RECORDS) {
- ctx.emit(v);
- }
- ctx.fireChannelInactive();
- }
- }
-
- public static class MockGroupDataSource implements DataSource {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
-
- public MockGroupDataSource() {
- for (int i = 0; i < 10; ++i) {
- RECORDS.add(new Values(i/3, i, (i+1)* 0.5, "x", i/2));
- }
- }
-
- @Override
- public void open(ChannelContext ctx) {
- for (Values v : RECORDS) {
- ctx.emit(v);
- }
- // force evaluation of the aggregate function on the last group
- ctx.flush();
- ctx.fireChannelInactive();
- }
- }
-
- public static class MockEmpDataSource implements DataSource {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
-
- public MockEmpDataSource() {
- RECORDS.add(new Values(1, "emp1", 1));
- RECORDS.add(new Values(2, "emp2", 1));
- RECORDS.add(new Values(3, "emp3", 2));
- }
-
- @Override
- public void open(ChannelContext ctx) {
- for (Values v : RECORDS) {
- ctx.emit(v);
- }
- ctx.flush();
- ctx.fireChannelInactive();
- }
- }
-
- public static class MockDeptDataSource implements DataSource {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
-
- public MockDeptDataSource() {
- RECORDS.add(new Values(1, "dept1"));
- RECORDS.add(new Values(2, "dept2"));
- RECORDS.add(new Values(3, "dept3"));
- }
-
- @Override
- public void open(ChannelContext ctx) {
- for (Values v : RECORDS) {
- ctx.emit(v);
- }
- ctx.flush();
- ctx.fireChannelInactive();
- }
- }
-
- public static class MockNestedDataSource implements DataSource {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
-
- public MockNestedDataSource() {
- List<Integer> ints = Arrays.asList(100, 200, 300);
- for (int i = 0; i < 5; ++i) {
- Map<String, Integer> map = new HashMap<>();
- map.put("b", i);
- map.put("c", i*i);
- Map<String, Map<String, Integer>> mm = new HashMap<>();
- mm.put("a", map);
- RECORDS.add(new Values(i, map, mm, ints));
- }
- }
-
- @Override
- public void open(ChannelContext ctx) {
- for (Values v : RECORDS) {
- ctx.emit(v);
- }
- ctx.fireChannelInactive();
- }
- }
-
- public static class MockState implements State {
- /**
- * Collect all values in a static variable as the instance will go through serialization and deserialization.
- * NOTE: This should be cleared before or after running each test.
- */
- private transient static final List<List<Object> > VALUES = new ArrayList<>();
-
- public static List<List<Object>> getCollectedValues() {
- return VALUES;
- }
-
- @Override
- public void beginCommit(Long txid) {
- // NOOP
- }
-
- @Override
- public void commit(Long txid) {
- // NOOP
- }
-
- public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
- for (TridentTuple tuple : tuples) {
- VALUES.add(tuple.getValues());
- }
- }
- }
-
- public static class MockStateFactory implements StateFactory {
-
- @Override
- public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
- return new MockState();
- }
- }
-
- public static class MockStateUpdater implements StateUpdater<MockState> {
-
- @Override
- public void updateState(MockState state, List<TridentTuple> tuples, TridentCollector collector) {
- state.updateState(tuples, collector);
- }
-
- @Override
- public void prepare(Map conf, TridentOperationContext context) {
- // NOOP
- }
-
- @Override
- public void cleanup() {
- // NOOP
- }
- }
-
- public static class MockSqlTridentDataSource implements ISqlTridentDataSource {
- @Override
- public IBatchSpout getProducer() {
- return new MockSpout();
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
- }
-
- private static class MockSpout implements IBatchSpout {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
- private final Fields OUTPUT_FIELDS = new Fields("ID", "NAME", "ADDR");
-
- public MockSpout() {
- RECORDS.add(new Values(0, "a", "y"));
- RECORDS.add(new Values(1, "ab", "y"));
- RECORDS.add(new Values(2, "abc", "y"));
- RECORDS.add(new Values(3, "abcd", "y"));
- RECORDS.add(new Values(4, "abcde", "y"));
- }
-
- private boolean emitted = false;
-
- @Override
- public void open(Map conf, TopologyContext context) {
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- if (emitted) {
- return;
- }
-
- for (Values r : RECORDS) {
- collector.emit(r);
- }
- emitted = true;
- }
-
- @Override
- public void ack(long batchId) {
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
- @Override
- public Fields getOutputFields() {
- return OUTPUT_FIELDS;
- }
- }
- }
-
- public static class MockSqlTridentGroupedDataSource implements ISqlTridentDataSource {
- @Override
- public IBatchSpout getProducer() {
- return new MockGroupedSpout();
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
- }
-
- private static class MockGroupedSpout implements IBatchSpout {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
- private final Fields OUTPUT_FIELDS = new Fields("ID", "GRPID", "NAME", "ADDR", "AGE", "SCORE");
-
- public MockGroupedSpout() {
- for (int i = 0; i < 5; ++i) {
- RECORDS.add(new Values(i, 0, "x", "y", 5 - i, i * 10));
- }
- }
-
- private boolean emitted = false;
-
- @Override
- public void open(Map conf, TopologyContext context) {
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- if (emitted) {
- return;
- }
-
- for (Values r : RECORDS) {
- collector.emit(r);
- }
- emitted = true;
- }
-
- @Override
- public void ack(long batchId) {
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
- @Override
- public Fields getOutputFields() {
- return OUTPUT_FIELDS;
- }
- }
- }
-
- public static class MockSqlTridentJoinDataSourceEmp implements ISqlTridentDataSource {
- @Override
- public IBatchSpout getProducer() {
- return new MockSpout();
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
- }
-
- private static class MockSpout implements IBatchSpout {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
- private final Fields OUTPUT_FIELDS = new Fields("EMPID", "EMPNAME", "DEPTID");
-
- public MockSpout() {
- for (int i = 0; i < 5; ++i) {
- RECORDS.add(new Values(i, "emp-" + i, i % 2));
- }
- for (int i = 10; i < 15; ++i) {
- RECORDS.add(new Values(i, "emp-" + i, i));
- }
- }
-
- private boolean emitted = false;
-
- @Override
- public void open(Map conf, TopologyContext context) {
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- if (emitted) {
- return;
- }
-
- for (Values r : RECORDS) {
- collector.emit(r);
- }
- emitted = true;
- }
-
- @Override
- public void ack(long batchId) {
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
- @Override
- public Fields getOutputFields() {
- return OUTPUT_FIELDS;
- }
- }
- }
-
- public static class MockSqlTridentJoinDataSourceDept implements ISqlTridentDataSource {
- @Override
- public IBatchSpout getProducer() {
- return new MockSpout();
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
- }
-
- private static class MockSpout implements IBatchSpout {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
- private final Fields OUTPUT_FIELDS = new Fields("DEPTID", "DEPTNAME");
-
- public MockSpout() {
- for (int i = 0; i < 5; ++i) {
- RECORDS.add(new Values(i, "dept-" + i));
- }
- }
-
- private boolean emitted = false;
-
- @Override
- public void open(Map conf, TopologyContext context) {
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- if (emitted) {
- return;
- }
-
- for (Values r : RECORDS) {
- collector.emit(r);
- }
- emitted = true;
- }
-
- @Override
- public void ack(long batchId) {
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
- @Override
- public Fields getOutputFields() {
- return OUTPUT_FIELDS;
- }
- }
- }
-
- public static class MockSqlTridentNestedDataSource implements ISqlTridentDataSource {
- @Override
- public IBatchSpout getProducer() {
- return new MockSpout();
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
- }
-
- private static class MockSpout implements IBatchSpout {
- private final ArrayList<Values> RECORDS = new ArrayList<>();
- private final Fields OUTPUT_FIELDS = new Fields("ID", "MAPFIELD", "NESTEDMAPFIELD", "ARRAYFIELD");
-
- public MockSpout() {
- List<Integer> ints = Arrays.asList(100, 200, 300);
- for (int i = 0; i < 5; ++i) {
- Map<String, Integer> map = new HashMap<>();
- map.put("b", i);
- map.put("c", i*i);
- Map<String, Map<String, Integer>> mm = new HashMap<>();
- mm.put("a", map);
- RECORDS.add(new Values(i, map, mm, ints));
- }
- }
-
- private boolean emitted = false;
-
- @Override
- public void open(Map conf, TopologyContext context) {
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- if (emitted) {
- return;
- }
-
- for (Values r : RECORDS) {
- collector.emit(r);
- }
- emitted = true;
- }
-
- @Override
- public void ack(long batchId) {
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
- @Override
- public Fields getOutputFields() {
- return OUTPUT_FIELDS;
- }
- }
- }
-
- public static class CollectDataChannelHandler implements ChannelHandler {
- private final List<Values> values;
-
- public CollectDataChannelHandler(List<Values> values) {
- this.values = values;
- }
-
- @Override
- public void dataReceived(ChannelContext ctx, Values data) {
- values.add(data);
- }
-
- @Override
- public void channelInactive(ChannelContext ctx) {}
-
- @Override
- public void exceptionCaught(Throwable cause) {
- throw new RuntimeException(cause);
- }
-
- @Override
- public void flush(ChannelContext ctx) {}
-
- @Override
- public void setSource(ChannelContext ctx, Object source) {}
- }
-
- public static long monotonicNow() {
- final long NANOSECONDS_PER_MILLISECOND = 1000000;
- return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
deleted file mode 100644
index 1e8edee..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.datasource.socket;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class TestSocketDataSourceProvider {
- private static final List<FieldInfo> FIELDS = ImmutableList.of(
- new FieldInfo("ID", int.class, true),
- new FieldInfo("val", String.class, false));
- private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
- private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
-
- @Test
- public void testSocketSink() throws IOException {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create("socket://localhost:8888"), null, null, new Properties(), FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(SocketState.Factory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(SocketStateUpdater.class, consumer.getStateUpdater().getClass());
-
- // makeState() fails on creating State so we just mock SocketState anyway
- SocketState mockState = mock(SocketState.class);
- StateUpdater stateUpdater = consumer.getStateUpdater();
-
- List<TridentTuple> tupleList = mockTupleList();
-
- stateUpdater.updateState(mockState, tupleList, null);
- for (TridentTuple t : tupleList) {
- String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
- verify(mockState).write(serializedValue + "\n");
- }
- }
-
- private static List<TridentTuple> mockTupleList() {
- List<TridentTuple> tupleList = new ArrayList<>();
- TridentTuple t0 = mock(TridentTuple.class);
- TridentTuple t1 = mock(TridentTuple.class);
- when(t0.getValueByField("ID")).thenReturn(1);
- when(t0.getValueByField("val")).thenReturn("2");
- doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
- when(t0.size()).thenReturn(2);
-
- when(t1.getValueByField("ID")).thenReturn(2);
- when(t1.getValueByField("val")).thenReturn("3");
- doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
- when(t1.size()).thenReturn(2);
-
- tupleList.add(t0);
- tupleList.add(t1);
- return tupleList;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/pom.xml b/external/storm-submit-tools/pom.xml
deleted file mode 100644
index 90e6dd8..0000000
--- a/external/storm-submit-tools/pom.xml
+++ /dev/null
@@ -1,216 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
- http://www.apache.org/licenses/LICENSE-2.0
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../pom.xml</relativePath>
- </parent>
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>storm-submit-tools</artifactId>
-
- <dependencies>
- <dependency>
- <groupId>commons-lang</groupId>
- <artifactId>commons-lang</artifactId>
- </dependency>
-
- <dependency>
- <groupId>com.googlecode.json-simple</groupId>
- <artifactId>json-simple</artifactId>
- </dependency>
-
- <dependency>
- <groupId>com.google.guava</groupId>
- <artifactId>guava</artifactId>
- </dependency>
-
- <!-- Aether :: maven dependency resolution -->
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-plugin-api</artifactId>
- <version>3.0</version>
- <exclusions>
- <exclusion>
- <groupId>org.codehaus.plexus</groupId>
- <artifactId>plexus-utils</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.sonatype.sisu</groupId>
- <artifactId>sisu-inject-plexus</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-model</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-api</artifactId>
- <version>1.12</version>
- </dependency>
-
- <dependency>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-util</artifactId>
- <version>1.12</version>
- </dependency>
-
- <dependency>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-impl</artifactId>
- <version>1.12</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.maven</groupId>
- <artifactId>maven-aether-provider</artifactId>
- <version>3.0.3</version>
- <exclusions>
- <exclusion>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-spi</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-util</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-impl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.plexus</groupId>
- <artifactId>plexus-utils</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-connector-file</artifactId>
- <version>1.12</version>
- </dependency>
-
- <dependency>
- <groupId>org.sonatype.aether</groupId>
- <artifactId>aether-connector-wagon</artifactId>
- <version>1.12</version>
- <exclusions>
- <exclusion>
- <groupId>org.apache.maven.wagon</groupId>
- <artifactId>wagon-provider-api</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.apache.maven.wagon</groupId>
- <artifactId>wagon-provider-api</artifactId>
- <version>1.0</version>
- <exclusions>
- <exclusion>
- <groupId>org.codehaus.plexus</groupId>
- <artifactId>plexus-utils</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.apache.maven.wagon</groupId>
- <artifactId>wagon-http-lightweight</artifactId>
- <version>1.0</version>
- <exclusions>
- <exclusion>
- <groupId>org.apache.maven.wagon</groupId>
- <artifactId>wagon-http-shared</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
-
- <dependency>
- <groupId>org.apache.maven.wagon</groupId>
- <artifactId>wagon-http</artifactId>
- <version>1.0</version>
- <exclusions>
- </exclusions>
- </dependency>
-
- <!-- storm-core is needed only for test (surefire) -->
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <configuration>
- <keepDependenciesWithProvidedScope>false</keepDependenciesWithProvidedScope>
- <createDependencyReducedPom>true</createDependencyReducedPom>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.sf</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.dsa</exclude>
- <exclude>META-INF/*.RSA</exclude>
- <exclude>META-INF/*.rsa</exclude>
- <exclude>META-INF/*.EC</exclude>
- <exclude>META-INF/*.ec</exclude>
- <exclude>META-INF/MSFTSIG.SF</exclude>
- <exclude>META-INF/MSFTSIG.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
- <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-
-</project>
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
deleted file mode 100644
index ac50110..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.submit.command;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import org.apache.commons.lang.StringUtils;
-import org.apache.storm.submit.dependency.AetherUtils;
-import org.apache.storm.submit.dependency.DependencyResolver;
-import org.json.simple.JSONValue;
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.resolution.ArtifactResult;
-
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-public class DependencyResolverMain {
-
- public static void main(String[] args) {
- if (args.length < 1) {
- throw new IllegalArgumentException("artifacts must be presented.");
- }
-
- String artifactsArg = args[0];
-
- // DO NOT CHANGE THIS TO SYSOUT
- System.err.println("DependencyResolver input - artifacts: " + artifactsArg);
- List<Dependency> dependencies = parseArtifactArgs(artifactsArg);
-
- List<RemoteRepository> repositories;
- if (args.length > 1) {
- String remoteRepositoryArg = args[1];
-
- // DO NOT CHANGE THIS TO SYSOUT
- System.err.println("DependencyResolver input - repositories: " + remoteRepositoryArg);
-
- repositories = parseRemoteRepositoryArgs(remoteRepositoryArg);
- } else {
- repositories = Collections.emptyList();
- }
-
- try {
- String localMavenRepoPath = getOrDefaultLocalMavenRepositoryPath("local-repo");
- DependencyResolver resolver = new DependencyResolver(localMavenRepoPath, repositories);
-
- List<ArtifactResult> artifactResults = resolver.resolve(dependencies);
-
- Iterable<ArtifactResult> missingArtifacts = filterMissingArtifacts(artifactResults);
- if (missingArtifacts.iterator().hasNext()) {
- printMissingArtifactsToSysErr(missingArtifacts);
- throw new RuntimeException("Some artifacts are not resolved");
- }
-
- System.out.println(JSONValue.toJSONString(transformArtifactResultToArtifactToPaths(artifactResults)));
- System.out.flush();
- } catch (Throwable e) {
- throw new RuntimeException(e);
- }
- }
-
- private static Iterable<ArtifactResult> filterMissingArtifacts(List<ArtifactResult> artifactResults) {
- return Iterables.filter(artifactResults, new Predicate<ArtifactResult>() {
- @Override
- public boolean apply(ArtifactResult artifactResult) {
- return artifactResult.isMissing();
- }
- });
- }
-
- private static void printMissingArtifactsToSysErr(Iterable<ArtifactResult> missingArtifacts) {
- for (ArtifactResult artifactResult : missingArtifacts) {
- System.err.println("ArtifactResult : " + artifactResult + " / Errors : " + artifactResult.getExceptions());
- }
- }
-
- private static List<Dependency> parseArtifactArgs(String artifactArgs) {
- List<String> artifacts = Arrays.asList(artifactArgs.split(","));
- List<Dependency> dependencies = new ArrayList<>(artifacts.size());
- for (String artifactOpt : artifacts) {
- if (artifactOpt.trim().isEmpty()) {
- continue;
- }
-
- dependencies.add(AetherUtils.parseDependency(artifactOpt));
- }
-
- return dependencies;
- }
-
- private static List<RemoteRepository> parseRemoteRepositoryArgs(String remoteRepositoryArg) {
- List<String> repositories = Arrays.asList(remoteRepositoryArg.split(","));
- List<RemoteRepository> remoteRepositories = new ArrayList<>(repositories.size());
- for (String repositoryOpt : repositories) {
- if (repositoryOpt.trim().isEmpty()) {
- continue;
- }
-
- remoteRepositories.add(AetherUtils.parseRemoteRepository(repositoryOpt));
- }
-
- return remoteRepositories;
- }
-
- private static Map<String, String> transformArtifactResultToArtifactToPaths(List<ArtifactResult> artifactResults) {
- Map<String, String> artifactToPath = new LinkedHashMap<>();
- for (ArtifactResult artifactResult : artifactResults) {
- Artifact artifact = artifactResult.getArtifact();
- artifactToPath.put(AetherUtils.artifactToString(artifact), artifact.getFile().getAbsolutePath());
- }
- return artifactToPath;
- }
-
- private static String getOrDefaultLocalMavenRepositoryPath(String defaultPath) {
- String localMavenRepoPathStr = getLocalMavenRepositoryPath();
- if (StringUtils.isNotEmpty(localMavenRepoPathStr)) {
- Path localMavenRepoPath = new File(localMavenRepoPathStr).toPath();
- if (Files.exists(localMavenRepoPath) && Files.isDirectory(localMavenRepoPath)) {
- return localMavenRepoPathStr;
- }
- }
-
- return defaultPath;
- }
-
- private static String getLocalMavenRepositoryPath() {
- String userHome = System.getProperty("user.home");
- if (StringUtils.isNotEmpty(userHome)) {
- return userHome + File.separator + ".m2" + File.separator + "repository";
- }
-
- return null;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
deleted file mode 100644
index 086be2b..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.submit.dependency;
-
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.Exclusion;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-public class AetherUtils {
- private AetherUtils() {
- }
-
- public static Dependency parseDependency(String dependency) {
- List<String> dependencyAndExclusions = Arrays.asList(dependency.split("\\^"));
- Collection<Exclusion> exclusions = new ArrayList<>();
- for (int idx = 1 ; idx < dependencyAndExclusions.size() ; idx++) {
- exclusions.add(AetherUtils.createExclusion(dependencyAndExclusions.get(idx)));
- }
-
- Artifact artifact = new DefaultArtifact(dependencyAndExclusions.get(0));
- return new Dependency(artifact, JavaScopes.COMPILE, false, exclusions);
- }
-
- public static Exclusion createExclusion(String exclusionString) {
- String[] parts = exclusionString.split(":");
-
- // length of parts should be greater than 0
- String groupId = parts[0];
-
- String artifactId = "*";
- String classifier = "*";
- String extension = "*";
-
- int len = parts.length;
- if (len > 1) {
- artifactId = parts[1];
- }
- if (len > 2) {
- classifier = parts[2];
- }
- if (len > 3) {
- extension = parts[3];
- }
-
- return new Exclusion(groupId, artifactId, classifier, extension);
- }
-
- public static String artifactToString(Artifact artifact) {
- StringBuilder buffer = new StringBuilder(128);
- buffer.append(artifact.getGroupId());
- buffer.append(':').append(artifact.getArtifactId());
- buffer.append(':').append(artifact.getExtension());
- if (artifact.getClassifier().length() > 0) {
- buffer.append(':').append(artifact.getClassifier());
- }
- buffer.append(':').append(artifact.getVersion());
- return buffer.toString();
- }
-
- public static RemoteRepository parseRemoteRepository(String repository) {
- String[] parts = repository.split("\\^");
- if (parts.length < 2) {
- throw new IllegalArgumentException("Bad remote repository form: " + repository);
- }
-
- return new RemoteRepository(parts[0], "default", parts[1]);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
deleted file mode 100644
index 816e3cc..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.submit.dependency;
-
-import org.apache.maven.repository.internal.MavenRepositorySystemSession;
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.RepositorySystemSession;
-import org.sonatype.aether.repository.LocalRepository;
-import org.sonatype.aether.repository.RemoteRepository;
-
-import java.io.File;
-
-/**
- * Manage mvn repository.
- */
-public class Booter {
- public static RepositorySystem newRepositorySystem() {
- return RepositorySystemFactory.newRepositorySystem();
- }
-
- public static RepositorySystemSession newRepositorySystemSession(
- RepositorySystem system, String localRepoPath) {
- MavenRepositorySystemSession session = new MavenRepositorySystemSession();
-
- LocalRepository localRepo =
- new LocalRepository(new File(localRepoPath).getAbsolutePath());
- session.setLocalRepositoryManager(system.newLocalRepositoryManager(localRepo));
-
- return session;
- }
-
- public static RemoteRepository newCentralRepository() {
- return new RemoteRepository("central", "default", "http://repo1.maven.org/maven2/");
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
deleted file mode 100644
index 4534344..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.submit.dependency;
-
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.RepositorySystemSession;
-import org.sonatype.aether.collection.CollectRequest;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.DependencyFilter;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.resolution.ArtifactResolutionException;
-import org.sonatype.aether.resolution.ArtifactResult;
-import org.sonatype.aether.resolution.DependencyRequest;
-import org.sonatype.aether.resolution.DependencyResolutionException;
-import org.sonatype.aether.util.artifact.JavaScopes;
-import org.sonatype.aether.util.filter.DependencyFilterUtils;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class DependencyResolver {
- private final RepositorySystem system = Booter.newRepositorySystem();
- private final RepositorySystemSession session;
-
- private final List<RemoteRepository> remoteRepositories;
-
- public DependencyResolver(String localRepoPath) {
- this(localRepoPath, Collections.emptyList());
- }
-
- public DependencyResolver(String localRepoPath, List<RemoteRepository> repositories) {
- localRepoPath = handleRelativePath(localRepoPath);
-
- session = Booter.newRepositorySystemSession(system, localRepoPath);
-
- remoteRepositories = new ArrayList<>();
- remoteRepositories.add(Booter.newCentralRepository());
- remoteRepositories.addAll(repositories);
- }
-
- private String handleRelativePath(String localRepoPath) {
- File repoDir = new File(localRepoPath);
- if (!repoDir.isAbsolute()) {
- // find homedir
- String home = System.getProperty("storm.home");
- if (home == null) {
- home = ".";
- }
-
- localRepoPath = home + "/" + localRepoPath;
- }
- return localRepoPath;
- }
-
- public List<ArtifactResult> resolve(List<Dependency> dependencies) throws MalformedURLException,
- DependencyResolutionException, ArtifactResolutionException {
-
- DependencyFilter classpathFilter = DependencyFilterUtils
- .classpathFilter(JavaScopes.COMPILE, JavaScopes.RUNTIME);
-
- if (dependencies.size() == 0) {
- return Collections.EMPTY_LIST;
- }
-
- CollectRequest collectRequest = new CollectRequest();
- collectRequest.setRoot(dependencies.get(0));
- for (int idx = 1; idx < dependencies.size(); idx++) {
- collectRequest.addDependency(dependencies.get(idx));
- }
-
- for (RemoteRepository repository : remoteRepositories) {
- collectRequest.addRepository(repository);
- }
-
- DependencyRequest dependencyRequest = new DependencyRequest(collectRequest, classpathFilter);
- return system.resolveDependencies(session, dependencyRequest).getArtifactResults();
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
deleted file mode 100644
index f1fa2db..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.submit.dependency;
-
-import org.apache.maven.repository.internal.DefaultServiceLocator;
-import org.apache.maven.wagon.Wagon;
-import org.apache.maven.wagon.providers.http.HttpWagon;
-import org.apache.maven.wagon.providers.http.LightweightHttpWagon;
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.connector.file.FileRepositoryConnectorFactory;
-import org.sonatype.aether.connector.wagon.WagonProvider;
-import org.sonatype.aether.connector.wagon.WagonRepositoryConnectorFactory;
-import org.sonatype.aether.spi.connector.RepositoryConnectorFactory;
-
-/**
- * Get maven repository instance.
- */
-public class RepositorySystemFactory {
- public static RepositorySystem newRepositorySystem() {
- DefaultServiceLocator locator = new DefaultServiceLocator();
- locator.addService(RepositoryConnectorFactory.class, FileRepositoryConnectorFactory.class);
- locator.addService(RepositoryConnectorFactory.class, WagonRepositoryConnectorFactory.class);
- locator.setServices(WagonProvider.class, new ManualWagonProvider());
-
- return locator.getService(RepositorySystem.class);
- }
-
- /**
- * ManualWagonProvider
- */
- public static class ManualWagonProvider implements WagonProvider {
-
- @Override
- public Wagon lookup(String roleHint) throws Exception {
- if ("http".equals(roleHint)) {
- return new LightweightHttpWagon();
- }
-
- if ("https".equals(roleHint)) {
- return new HttpWagon();
- }
-
- return null;
- }
-
- @Override
- public void release(Wagon arg0) {
-
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java b/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
deleted file mode 100644
index 9951621..0000000
--- a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.submit.dependency;
-
-import com.google.common.collect.Lists;
-import org.junit.Test;
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.Exclusion;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-public class AetherUtilsTest {
- @Test
- public void parseDependency() throws Exception {
- String testDependency = "testgroup:testartifact:1.0.0^testgroup:testexcartifact^testgroup:*";
-
- Dependency dependency = AetherUtils.parseDependency(testDependency);
-
- assertEquals("testgroup", dependency.getArtifact().getGroupId());
- assertEquals("testartifact", dependency.getArtifact().getArtifactId());
- assertEquals("1.0.0", dependency.getArtifact().getVersion());
- assertEquals(JavaScopes.COMPILE, dependency.getScope());
-
- assertEquals(2, dependency.getExclusions().size());
-
- List<Exclusion> exclusions = Lists.newArrayList(dependency.getExclusions());
-
- Exclusion exclusion = exclusions.get(0);
- assertEquals("testgroup", exclusion.getGroupId());
- assertEquals("testexcartifact", exclusion.getArtifactId());
- assertEquals(JavaScopes.COMPILE, dependency.getScope());
-
- exclusion = exclusions.get(1);
- assertEquals("testgroup", exclusion.getGroupId());
- assertEquals("*", exclusion.getArtifactId());
- assertEquals(JavaScopes.COMPILE, dependency.getScope());
- }
-
- @Test
- public void createExclusion() throws Exception {
- String testExclusion = "group";
- Exclusion exclusion = AetherUtils.createExclusion(testExclusion);
-
- assertEquals("group", exclusion.getGroupId());
- assertEquals("*", exclusion.getArtifactId());
- assertEquals("*", exclusion.getClassifier());
- assertEquals("*", exclusion.getExtension());
-
- testExclusion = "group:artifact";
- exclusion = AetherUtils.createExclusion(testExclusion);
-
- assertEquals("group", exclusion.getGroupId());
- assertEquals("artifact", exclusion.getArtifactId());
- assertEquals("*", exclusion.getClassifier());
- assertEquals("*", exclusion.getExtension());
-
- testExclusion = "group:artifact:site";
- exclusion = AetherUtils.createExclusion(testExclusion);
-
- assertEquals("group", exclusion.getGroupId());
- assertEquals("artifact", exclusion.getArtifactId());
- assertEquals("site", exclusion.getClassifier());
- assertEquals("*", exclusion.getExtension());
-
- testExclusion = "group:artifact:site:jar";
- exclusion = AetherUtils.createExclusion(testExclusion);
-
- assertEquals("group", exclusion.getGroupId());
- assertEquals("artifact", exclusion.getArtifactId());
- assertEquals("site", exclusion.getClassifier());
- assertEquals("jar", exclusion.getExtension());
- }
-
- @Test
- public void artifactToString() throws Exception {
- Artifact testArtifact = new DefaultArtifact("org.apache.storm:storm-core:1.0.0");
-
- String ret = AetherUtils.artifactToString(testArtifact);
- assertEquals("org.apache.storm:storm-core:jar:1.0.0", ret);
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java b/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
deleted file mode 100644
index 086abad..0000000
--- a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.submit.dependency;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.FileUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.resolution.ArtifactResult;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
-public class DependencyResolverTest {
- private static Path tempDirForTest;
-
- private DependencyResolver sut;
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
- tempDirForTest = Files.createTempDirectory("dr-test");
- }
-
- @AfterClass
- public static void tearDownAfterClass() throws Exception {
- FileUtils.deleteQuietly(tempDirForTest.toFile());
- }
-
- @Before
- public void setUp() {
- sut = new DependencyResolver(tempDirForTest.toAbsolutePath().toString());
- }
-
- @Test
- public void resolveValid() throws Exception {
- // please pick small artifact which has small transitive dependency
- // and let's mark as Ignore if we want to run test even without internet or maven central is often not stable
- Dependency dependency = new Dependency(new DefaultArtifact("org.apache.storm:flux-core:1.0.0"), JavaScopes.COMPILE);
- List<ArtifactResult> results = sut.resolve(Lists.newArrayList(dependency));
-
- assertTrue(results.size() > 0);
- // it should be org.apache.storm:flux-core:jar:1.0.0 and commons-cli:commons-cli:jar:1.2
- assertContains(results, "org.apache.storm", "flux-core", "1.0.0");
- assertContains(results, "commons-cli", "commons-cli", "1.2");
- }
-
- private void assertContains(List<ArtifactResult> results, String groupId, String artifactId, String version) {
- for (ArtifactResult result : results) {
- if (result.getArtifact().getGroupId().equals(groupId) &&
- result.getArtifact().getArtifactId().equals(artifactId) &&
- result.getArtifact().getVersion().equals(version) &&
- result.isResolved()) {
- return;
- }
- }
-
- throw new AssertionError("Result doesn't contain expected artifact > " + groupId + ":" + artifactId + ":" + version);
- }
-
-}
\ No newline at end of file
[03/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
new file mode 100644
index 0000000..38c3fcb
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.hdfs;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.hdfs.trident.HdfsState;
+import org.apache.storm.hdfs.trident.HdfsStateFactory;
+import org.apache.storm.hdfs.trident.HdfsUpdater;
+import org.apache.storm.hdfs.trident.format.FileNameFormat;
+import org.apache.storm.hdfs.trident.format.RecordFormat;
+import org.apache.storm.hdfs.trident.format.SimpleFileNameFormat;
+import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.trident.rotation.TimedRotationPolicy;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a HDFS sink based on the URI and properties. The URI has the format of hdfs://host:port/path-to-file
+ * The properties are in JSON format which specifies the name / path of the hdfs file and etc.
+ */
+public class HdfsDataSourcesProvider implements DataSourcesProvider {
+
+ private static class HdfsTridentDataSource implements ISqlTridentDataSource {
+ private final String url;
+ private final Properties props;
+ private final IOutputSerializer serializer;
+
+ private HdfsTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
+ this.url = url;
+ this.props = props;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public ITridentDataSource getProducer() {
+ throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ FileNameFormat fileNameFormat = new SimpleFileNameFormat()
+ .withPath(props.getProperty("hdfs.file.path", "/storm"))
+ .withName(props.getProperty("hdfs.file.name", "$TIME.$NUM.txt"));
+
+ RecordFormat recordFormat = new TridentRecordFormat(serializer);
+
+ FileRotationPolicy rotationPolicy;
+ String size = props.getProperty("hdfs.rotation.size.kb");
+ String interval = props.getProperty("hdfs.rotation.time.seconds");
+ Preconditions.checkArgument(size != null || interval != null, "Hdfs data source must contain file rotation config");
+
+ if (size != null) {
+ rotationPolicy = new FileSizeRotationPolicy(Float.parseFloat(size), FileSizeRotationPolicy.Units.KB);
+ } else {
+ rotationPolicy = new TimedRotationPolicy(Float.parseFloat(interval), TimedRotationPolicy.TimeUnit.SECONDS);
+ }
+
+ HdfsState.Options options = new HdfsState.HdfsFileOptions()
+ .withFileNameFormat(fileNameFormat)
+ .withRecordFormat(recordFormat)
+ .withRotationPolicy(rotationPolicy)
+ .withFsUrl(url);
+
+ StateFactory stateFactory = new HdfsStateFactory().withOptions(options);
+ StateUpdater stateUpdater = new HdfsUpdater();
+
+ return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+ }
+ }
+
+ private static class TridentRecordFormat implements RecordFormat {
+ private final IOutputSerializer serializer;
+
+ private TridentRecordFormat(IOutputSerializer serializer) {
+ this.serializer = serializer;
+ }
+
+ @Override
+ public byte[] format(TridentTuple tuple) {
+ //TODO we should handle '\n'. ref DelimitedRecordFormat
+ return serializer.write(tuple.getValues(), null).array();
+ }
+
+ }
+
+ @Override
+ public String scheme() {
+ return "hdfs";
+ }
+
+ @Override
+ public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+ IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+ return new HdfsTridentDataSource(uri.toString(), properties, serializer);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..5fac84f
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.storm.sql.hdfs.HdfsDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
new file mode 100644
index 0000000..1473438
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.hdfs;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.storm.hdfs.trident.HdfsState;
+import org.apache.storm.hdfs.trident.HdfsStateFactory;
+import org.apache.storm.hdfs.trident.HdfsUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.storm.hdfs.trident.HdfsState.HdfsFileOptions;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class TestHdfsDataSourcesProvider {
+ private static final List<FieldInfo> FIELDS = ImmutableList.of(
+ new FieldInfo("ID", int.class, true),
+ new FieldInfo("val", String.class, false));
+ private static final Properties TBL_PROPERTIES = new Properties();
+
+ private static String hdfsURI;
+ private static MiniDFSCluster hdfsCluster;
+
+ static {
+ TBL_PROPERTIES.put("hdfs.file.path", "/unittest");
+ TBL_PROPERTIES.put("hdfs.file.name", "test1.txt");
+ TBL_PROPERTIES.put("hdfs.rotation.time.seconds", "120");
+ }
+
+ @Before
+ public void setup() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set("fs.trash.interval", "10");
+ conf.setBoolean("dfs.permissions", true);
+ File baseDir = new File("./target/hdfs/").getAbsoluteFile();
+ FileUtil.fullyDelete(baseDir);
+ conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+ hdfsCluster = builder.build();
+ hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
+ }
+
+ @After
+ public void shutDown() throws IOException {
+ hdfsCluster.shutdown();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testHdfsSink() {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create(hdfsURI), null, null, TBL_PROPERTIES, FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(HdfsStateFactory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(HdfsUpdater.class, consumer.getStateUpdater().getClass());
+
+ HdfsState state = (HdfsState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+ StateUpdater stateUpdater = consumer.getStateUpdater();
+
+ HdfsFileOptions options = mock(HdfsFileOptions.class);
+ Whitebox.setInternalState(state, "options", options);
+
+ List<TridentTuple> tupleList = mockTupleList();
+
+ for (TridentTuple t : tupleList) {
+ stateUpdater.updateState(state, Collections.singletonList(t), null);
+ try {
+ verify(options).execute(Collections.singletonList(t));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ private static List<TridentTuple> mockTupleList() {
+ List<TridentTuple> tupleList = new ArrayList<>();
+ TridentTuple t0 = mock(TridentTuple.class);
+ TridentTuple t1 = mock(TridentTuple.class);
+ doReturn(1).when(t0).get(0);
+ doReturn(2).when(t1).get(0);
+ doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+ doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+ tupleList.add(t0);
+ tupleList.add(t1);
+ return tupleList;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/pom.xml b/sql/storm-sql-external/storm-sql-kafka/pom.xml
new file mode 100644
index 0000000..881343d
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-kafka</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-kafka</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>${storm.kafka.artifact.id}</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>kafka-clients</artifactId>
+ <version>${storm.kafka.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
new file mode 100644
index 0000000..2aa98ba
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.kafka;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.kafka.ZkHosts;
+import org.apache.storm.kafka.trident.OpaqueTridentKafkaSpout;
+import org.apache.storm.kafka.trident.TridentKafkaConfig;
+import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
+import org.apache.storm.kafka.trident.TridentKafkaUpdater;
+import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
+import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.spout.SchemeAsMultiScheme;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Create a Kafka spout/sink based on the URI and properties. The URI has the format of
+ * kafka://zkhost:port/broker_path?topic=topic. The properties are in JSON format which specifies the producer config
+ * of the Kafka broker.
+ */
+public class KafkaDataSourcesProvider implements DataSourcesProvider {
+ private static final int DEFAULT_ZK_PORT = 2181;
+
+ private static class SqlKafkaMapper implements TridentTupleToKafkaMapper<Object, ByteBuffer> {
+ private final int primaryKeyIndex;
+ private final IOutputSerializer serializer;
+
+ private SqlKafkaMapper(int primaryKeyIndex, IOutputSerializer serializer) {
+ this.primaryKeyIndex = primaryKeyIndex;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public Object getKeyFromTuple(TridentTuple tuple) {
+ return tuple.get(primaryKeyIndex);
+ }
+
+ @Override
+ public ByteBuffer getMessageFromTuple(TridentTuple tuple) {
+ return serializer.write(tuple.getValues(), null);
+ }
+ }
+
+ private static class KafkaTridentDataSource implements ISqlTridentDataSource {
+ private final TridentKafkaConfig conf;
+ private final String topic;
+ private final int primaryKeyIndex;
+ private final Properties props;
+ private final IOutputSerializer serializer;
+ private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
+ Properties props, IOutputSerializer serializer) {
+ this.conf = conf;
+ this.topic = topic;
+ this.primaryKeyIndex = primaryKeyIndex;
+ this.props = props;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public ITridentDataSource getProducer() {
+ return new OpaqueTridentKafkaSpout(conf);
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ Preconditions.checkArgument(!props.isEmpty(),
+ "Writable Kafka Table " + topic + " must contain producer config");
+ HashMap<String, Object> producerConfig = (HashMap<String, Object>) props.get("producer");
+ props.putAll(producerConfig);
+ Preconditions.checkState(props.containsKey("bootstrap.servers"),
+ "Writable Kafka Table " + topic + " must contain \"bootstrap.servers\" config");
+
+ SqlKafkaMapper mapper = new SqlKafkaMapper(primaryKeyIndex, serializer);
+
+ TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
+ .withKafkaTopicSelector(new DefaultTopicSelector(topic))
+ .withProducerProperties(props)
+ .withTridentTupleToKafkaMapper(mapper);
+
+ TridentKafkaUpdater stateUpdater = new TridentKafkaUpdater();
+
+ return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+ }
+ }
+
+ @Override
+ public String scheme() {
+ return "kafka";
+ }
+
+ @Override
+ public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_ZK_PORT;
+ ZkHosts zk = new ZkHosts(uri.getHost() + ":" + port, uri.getPath());
+ Map<String, String> values = parseURIParams(uri.getQuery());
+ String topic = values.get("topic");
+ Preconditions.checkNotNull(topic, "No topic of the spout is specified");
+ TridentKafkaConfig conf = new TridentKafkaConfig(zk, topic);
+ 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);
+ conf.scheme = new SchemeAsMultiScheme(scheme);
+ IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+
+ return new KafkaTridentDataSource(conf, topic, primaryIndex, properties, serializer);
+ }
+
+ private static Map<String, String> parseURIParams(String query) {
+ HashMap<String, String> res = new HashMap<>();
+ if (query == null) {
+ return res;
+ }
+
+ String[] params = query.split("&");
+ for (String p : params) {
+ String[] v = p.split("=", 2);
+ if (v.length > 1) {
+ res.put(v[0], v[1]);
+ }
+ }
+ return res;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..7f687cc
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.storm.sql.kafka.KafkaDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
new file mode 100644
index 0000000..0cde492
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.kafka;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.storm.kafka.trident.TridentKafkaState;
+import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
+import org.apache.storm.kafka.trident.TridentKafkaUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.Future;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+public class TestKafkaDataSourcesProvider {
+ private static final List<FieldInfo> FIELDS = ImmutableList.of(
+ new FieldInfo("ID", int.class, true),
+ new FieldInfo("val", String.class, false));
+ private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+ private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+ private static final Properties TBL_PROPERTIES = new Properties();
+
+ static {
+ Map<String,Object> map = new HashMap<>();
+ map.put("bootstrap.servers", "localhost:9092");
+ map.put("acks", "1");
+ map.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ map.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ TBL_PROPERTIES.put("producer", map);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testKafkaSink() {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create("kafka://mock?topic=foo"), null, null, TBL_PROPERTIES, FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(TridentKafkaStateFactory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(TridentKafkaUpdater.class, consumer.getStateUpdater().getClass());
+
+ TridentKafkaState state = (TridentKafkaState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+ KafkaProducer producer = mock(KafkaProducer.class);
+ doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class));
+ Whitebox.setInternalState(state, "producer", producer);
+
+ List<TridentTuple> tupleList = mockTupleList();
+ for (TridentTuple t : tupleList) {
+ state.updateState(Collections.singletonList(t), null);
+ verify(producer).send(argThat(new KafkaMessageMatcher(t)));
+ }
+ verifyNoMoreInteractions(producer);
+ }
+
+ private static List<TridentTuple> mockTupleList() {
+ List<TridentTuple> tupleList = new ArrayList<>();
+ TridentTuple t0 = mock(TridentTuple.class);
+ TridentTuple t1 = mock(TridentTuple.class);
+ doReturn(1).when(t0).get(0);
+ doReturn(2).when(t1).get(0);
+ doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+ doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+ tupleList.add(t0);
+ tupleList.add(t1);
+ return tupleList;
+ }
+
+ private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
+ private static final int PRIMARY_INDEX = 0;
+ private final TridentTuple tuple;
+
+ private KafkaMessageMatcher(TridentTuple tuple) {
+ this.tuple = tuple;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public boolean matches(Object o) {
+ ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
+ if (m.key() != tuple.get(PRIMARY_INDEX)) {
+ return false;
+ }
+ ByteBuffer buf = m.value();
+ ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
+ return b.equals(buf);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/pom.xml b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
new file mode 100644
index 0000000..218d89b
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
@@ -0,0 +1,84 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-mongodb</artifactId>
+
+ <developers>
+ <developer>
+ <id>vesense</id>
+ <name>Xin Wang</name>
+ <email>data.xinwang@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-mongodb</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
new file mode 100644
index 0000000..60d52d1
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.mongodb;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.mongodb.common.mapper.MongoMapper;
+import org.apache.storm.mongodb.trident.state.MongoState;
+import org.apache.storm.mongodb.trident.state.MongoStateFactory;
+import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.tuple.ITuple;
+import org.bson.Document;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a MongoDB sink based on the URI and properties. The URI has the format of
+ * mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]][/[database][?options]].
+ * The properties are in JSON format which specifies the name of the MongoDB collection and etc.
+ */
+public class MongoDataSourcesProvider implements DataSourcesProvider {
+
+ private static class MongoTridentDataSource implements ISqlTridentDataSource {
+ private final String url;
+ private final Properties props;
+ private final IOutputSerializer serializer;
+
+ private MongoTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
+ this.url = url;
+ this.props = props;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public ITridentDataSource getProducer() {
+ throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ Preconditions.checkArgument(!props.isEmpty(), "Writable MongoDB must contain collection config");
+ String serField = props.getProperty("trident.ser.field", "tridentSerField");
+ MongoMapper mapper = new TridentMongoMapper(serField, serializer);
+
+ MongoState.Options options = new MongoState.Options()
+ .withUrl(url)
+ .withCollectionName(props.getProperty("collection.name"))
+ .withMapper(mapper);
+
+ StateFactory stateFactory = new MongoStateFactory(options);
+ StateUpdater stateUpdater = new MongoStateUpdater();
+
+ return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+ }
+ }
+
+ private static class TridentMongoMapper implements MongoMapper {
+ private final String serField;
+ private final IOutputSerializer serializer;
+
+ private TridentMongoMapper(String serField, IOutputSerializer serializer) {
+ this.serField = serField;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public Document toDocument(ITuple tuple) {
+ Document document = new Document();
+ byte[] array = serializer.write(tuple.getValues(), null).array();
+ document.append(serField, array);
+ return document;
+ }
+
+ @Override
+ public Document toDocumentByKeys(List<Object> keys) {
+ return null;
+ }
+ }
+
+ @Override
+ public String scheme() {
+ return "mongodb";
+ }
+
+ @Override
+ public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+ IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+ return new MongoTridentDataSource(uri.toString(), properties, serializer);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..e46d794
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.storm.sql.mongodb.MongoDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
new file mode 100644
index 0000000..3b15345
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.mongodb;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.mongodb.common.MongoDBClient;
+import org.apache.storm.mongodb.trident.state.MongoState;
+import org.apache.storm.mongodb.trident.state.MongoStateFactory;
+import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.bson.Document;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+public class TestMongoDataSourcesProvider {
+ private static final List<FieldInfo> FIELDS = ImmutableList.of(
+ new FieldInfo("ID", int.class, true),
+ new FieldInfo("val", String.class, false));
+ private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+ private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+ private static final Properties TBL_PROPERTIES = new Properties();
+
+ static {
+ TBL_PROPERTIES.put("collection.name", "collection1");
+ TBL_PROPERTIES.put("trident.ser.field", "tridentSerField");
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testMongoSink() {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create("mongodb://127.0.0.1:27017/test"), null, null, TBL_PROPERTIES, FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(MongoStateFactory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(MongoStateUpdater.class, consumer.getStateUpdater().getClass());
+
+ MongoState state = (MongoState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+ StateUpdater stateUpdater = consumer.getStateUpdater();
+
+ MongoDBClient mongoClient = mock(MongoDBClient.class);
+ Whitebox.setInternalState(state, "mongoClient", mongoClient);
+
+ List<TridentTuple> tupleList = mockTupleList();
+
+ for (TridentTuple t : tupleList) {
+ stateUpdater.updateState(state, Collections.singletonList(t), null);
+ verify(mongoClient).insert(argThat(new MongoArgMatcher(t)) , eq(true));
+ }
+
+ verifyNoMoreInteractions(mongoClient);
+ }
+
+ private static List<TridentTuple> mockTupleList() {
+ List<TridentTuple> tupleList = new ArrayList<>();
+ TridentTuple t0 = mock(TridentTuple.class);
+ TridentTuple t1 = mock(TridentTuple.class);
+ doReturn(1).when(t0).get(0);
+ doReturn(2).when(t1).get(0);
+ doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+ doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+ tupleList.add(t0);
+ tupleList.add(t1);
+ return tupleList;
+ }
+
+ private static class MongoArgMatcher extends ArgumentMatcher<List<Document>> {
+ private final TridentTuple tuple;
+
+ private MongoArgMatcher(TridentTuple tuple) {
+ this.tuple = tuple;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public boolean matches(Object o) {
+ Document doc = ((List<Document>)o).get(0);
+ ByteBuffer buf = ByteBuffer.wrap((byte[])doc.get(TBL_PROPERTIES.getProperty("trident.ser.field")));
+ ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
+ return b.equals(buf);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/pom.xml b/sql/storm-sql-external/storm-sql-redis/pom.xml
new file mode 100644
index 0000000..32ce432
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/pom.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-redis</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-redis</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
new file mode 100644
index 0000000..68933b2
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.redis;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.redis.common.config.JedisClusterConfig;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
+import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
+import org.apache.storm.redis.common.mapper.RedisStoreMapper;
+import org.apache.storm.redis.trident.state.RedisClusterState;
+import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
+import org.apache.storm.redis.trident.state.RedisState;
+import org.apache.storm.redis.trident.state.RedisStateUpdater;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.tuple.ITuple;
+import redis.clients.util.JedisURIHelper;
+
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a Redis sink based on the URI and properties. The URI has the format of
+ * redis://:[password]@[host]:[port]/[dbIdx]. Only host is mandatory and others can be set to default.
+ *
+ * The properties are in JSON format which specifies the config of the Redis data type and etc.
+ * Please note that when "use.redis.cluster" is "true", cluster discovery is only done from given URI.
+ */
+public class RedisDataSourcesProvider implements DataSourcesProvider {
+ private static final int DEFAULT_REDIS_PORT = 6379;
+ private static final int DEFAULT_TIMEOUT = 2000;
+
+ private abstract static class AbstractRedisTridentDataSource implements ISqlTridentDataSource, Serializable {
+ protected abstract StateFactory newStateFactory();
+ protected abstract StateUpdater newStateUpdater(RedisStoreMapper storeMapper);
+
+ private final Properties props;
+ private final List<FieldInfo> fields;
+ private final IOutputSerializer serializer;
+
+ AbstractRedisTridentDataSource(Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+ this.props = props;
+ this.fields = fields;
+ this.serializer = serializer;
+ }
+
+ @Override
+ public ITridentDataSource getProducer() {
+ throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ RedisDataTypeDescription dataTypeDescription = getDataTypeDesc(props);
+
+ RedisStoreMapper storeMapper = new TridentRedisStoreMapper(dataTypeDescription, fields, serializer);
+
+ StateFactory stateFactory = newStateFactory();
+ StateUpdater stateUpdater = newStateUpdater(storeMapper);
+
+ return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+ }
+
+ private RedisDataTypeDescription getDataTypeDesc(Properties props) {
+ Preconditions.checkArgument(props.containsKey("data.type"),
+ "Redis data source must contain \"data.type\" config");
+
+ RedisDataTypeDescription.RedisDataType dataType = RedisDataTypeDescription.RedisDataType.valueOf(props.getProperty("data.type").toUpperCase());
+ String additionalKey = props.getProperty("data.additional.key");
+
+ return new RedisDataTypeDescription(dataType, additionalKey);
+ }
+ }
+
+ private static class RedisClusterTridentDataSource extends AbstractRedisTridentDataSource {
+ private final JedisClusterConfig config;
+
+ RedisClusterTridentDataSource(JedisClusterConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+ super(props, fields, serializer);
+ this.config = config;
+ }
+
+ @Override
+ protected StateFactory newStateFactory() {
+ return new RedisClusterState.Factory(config);
+ }
+
+ @Override
+ protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
+ return new RedisClusterStateUpdater(storeMapper);
+ }
+ }
+
+ private static class RedisTridentDataSource extends AbstractRedisTridentDataSource {
+ private final JedisPoolConfig config;
+
+ RedisTridentDataSource(JedisPoolConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+ super(props, fields, serializer);
+ this.config = config;
+ }
+
+ @Override
+ protected StateFactory newStateFactory() {
+ return new RedisState.Factory(config);
+ }
+
+ @Override
+ protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
+ return new RedisStateUpdater(storeMapper);
+ }
+ }
+
+ @Override
+ public String scheme() {
+ return "redis";
+ }
+
+ @Override
+ public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties props, List<FieldInfo> fields) {
+ Preconditions.checkArgument(JedisURIHelper.isValid(uri), "URI is not valid for Redis: " + uri);
+
+ String host = uri.getHost();
+ int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_REDIS_PORT;
+ int dbIdx = JedisURIHelper.getDBIndex(uri);
+ String password = JedisURIHelper.getPassword(uri);
+
+ int timeout = Integer.parseInt(props.getProperty("redis.timeout", String.valueOf(DEFAULT_TIMEOUT)));
+
+ boolean clusterMode = Boolean.valueOf(props.getProperty("use.redis.cluster", "false"));
+
+ List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+ IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, props, fieldNames);
+ if (clusterMode) {
+ JedisClusterConfig config = new JedisClusterConfig.Builder()
+ .setNodes(Collections.singleton(new InetSocketAddress(host, port)))
+ .setTimeout(timeout)
+ .build();
+ return new RedisClusterTridentDataSource(config, props, fields, serializer);
+ } else {
+ JedisPoolConfig config = new JedisPoolConfig(host, port, timeout, password, dbIdx);
+ return new RedisTridentDataSource(config, props, fields, serializer);
+ }
+ }
+
+ private static class TridentRedisStoreMapper implements RedisStoreMapper {
+ private final RedisDataTypeDescription dataTypeDescription;
+ private final FieldInfo primaryKeyField;
+ private final IOutputSerializer outputSerializer;
+
+ private TridentRedisStoreMapper(RedisDataTypeDescription dataTypeDescription, List<FieldInfo> fields, IOutputSerializer outputSerializer) {
+ this.dataTypeDescription = dataTypeDescription;
+ this.outputSerializer = outputSerializer;
+
+ // find primary key from constructor
+ FieldInfo pkField = findPrimaryKeyField(fields);
+ Preconditions.checkArgument(pkField != null, "Primary key must be presented to field list");
+
+ this.primaryKeyField = pkField;
+ }
+
+ 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 RedisDataTypeDescription getDataTypeDescription() {
+ return dataTypeDescription;
+ }
+
+ @Override
+ public String getKeyFromTuple(ITuple tuple) {
+ String keyFieldName = primaryKeyField.name();
+ Object key = tuple.getValueByField(keyFieldName);
+ if (key == null) {
+ throw new NullPointerException("key field " + keyFieldName + " is null");
+ }
+ return String.valueOf(key);
+ }
+
+ @Override
+ public String getValueFromTuple(ITuple tuple) {
+ byte[] array = outputSerializer.write(tuple.getValues(), null).array();
+ return new String(array);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..23b0444
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.storm.sql.redis.RedisDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
new file mode 100644
index 0000000..94d4949
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.redis;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.redis.trident.state.RedisClusterState;
+import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
+import org.apache.storm.redis.trident.state.RedisState;
+import org.apache.storm.redis.trident.state.RedisStateUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.Pipeline;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestRedisDataSourcesProvider {
+ private static final List<FieldInfo> FIELDS = ImmutableList.of(
+ new FieldInfo("ID", int.class, true),
+ new FieldInfo("val", String.class, false));
+ private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+ private static final String ADDITIONAL_KEY = "hello";
+ private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+ private static final Properties TBL_PROPERTIES = new Properties();
+ private static final Properties CLUSTER_TBL_PROPERTIES = new Properties();
+
+ static {
+ TBL_PROPERTIES.put("data.type", "HASH");
+ TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
+ CLUSTER_TBL_PROPERTIES.put("data.type", "HASH");
+ CLUSTER_TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
+ CLUSTER_TBL_PROPERTIES.put("use.redis.cluster", "true");
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testRedisSink() {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create("redis://:foobared@localhost:6380/2"), null, null, TBL_PROPERTIES, FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(RedisState.Factory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(RedisStateUpdater.class, consumer.getStateUpdater().getClass());
+
+ RedisState state = (RedisState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+ StateUpdater stateUpdater = consumer.getStateUpdater();
+
+ JedisPool mockJedisPool = mock(JedisPool.class);
+ Jedis mockJedis = mock(Jedis.class);
+ Pipeline mockPipeline = mock(Pipeline.class);
+
+ Whitebox.setInternalState(state, "jedisPool", mockJedisPool);
+ when(mockJedisPool.getResource()).thenReturn(mockJedis);
+ when(mockJedis.pipelined()).thenReturn(mockPipeline);
+
+ List<TridentTuple> tupleList = mockTupleList();
+
+ stateUpdater.updateState(state, tupleList, null);
+ for (TridentTuple t : tupleList) {
+ // PK goes to the key
+ String id = String.valueOf(t.getValueByField("ID"));
+ String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+ verify(mockPipeline).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
+ }
+
+ verify(mockPipeline).sync();
+ verify(mockJedis).close();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testRedisClusterSink() throws IOException {
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+ URI.create("redis://localhost:6380"), null, null, CLUSTER_TBL_PROPERTIES, FIELDS);
+ Assert.assertNotNull(ds);
+
+ ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+ Assert.assertEquals(RedisClusterState.Factory.class, consumer.getStateFactory().getClass());
+ Assert.assertEquals(RedisClusterStateUpdater.class, consumer.getStateUpdater().getClass());
+
+ RedisClusterState state = (RedisClusterState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+ StateUpdater stateUpdater = consumer.getStateUpdater();
+
+ JedisCluster mockJedisCluster = mock(JedisCluster.class);
+
+ Whitebox.setInternalState(state, "jedisCluster", mockJedisCluster);
+
+ List<TridentTuple> tupleList = mockTupleList();
+
+ stateUpdater.updateState(state, tupleList, null);
+ for (TridentTuple t : tupleList) {
+ // PK goes to the key
+ String id = String.valueOf(t.getValueByField("ID"));
+ String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+ verify(mockJedisCluster).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
+ }
+
+ verify(mockJedisCluster, never()).close();
+ }
+
+ private static List<TridentTuple> mockTupleList() {
+ List<TridentTuple> tupleList = new ArrayList<>();
+ TridentTuple t0 = mock(TridentTuple.class);
+ TridentTuple t1 = mock(TridentTuple.class);
+ when(t0.getValueByField("ID")).thenReturn(1);
+ when(t0.getValueByField("val")).thenReturn("2");
+ doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+
+ when(t1.getValueByField("ID")).thenReturn(2);
+ when(t1.getValueByField("val")).thenReturn("3");
+ doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+
+ tupleList.add(t0);
+ tupleList.add(t1);
+ return tupleList;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/pom.xml b/sql/storm-sql-runtime/pom.xml
new file mode 100644
index 0000000..ce57cb2
--- /dev/null
+++ b/sql/storm-sql-runtime/pom.xml
@@ -0,0 +1,136 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-runtime</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <version>${calcite.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>commons-dbcp</groupId>
+ <artifactId>commons-dbcp</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.pentaho</groupId>
+ <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro</artifactId>
+ <version>1.7.7</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-csv</artifactId>
+ <version>1.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.9</version>
+ <executions>
+ <execution>
+ <id>create-repo</id>
+ <goals>
+ <goal>create-repository</goal>
+ </goals>
+ <configuration>
+ <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+ <repositoryLayout>flat</repositoryLayout>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
new file mode 100644
index 0000000..aa7e435
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.interpreter;
+
+import org.apache.calcite.DataContext;
+
+import java.io.Serializable;
+
+/**
+ * This is a hack to use Calcite Context.
+ */
+public class StormContext extends Context implements Serializable {
+ public StormContext(DataContext root) {
+ super(root);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
new file mode 100644
index 0000000..64be39d
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public abstract class AbstractChannelHandler implements ChannelHandler {
+ @Override
+ public abstract void dataReceived(ChannelContext ctx, Values data);
+
+ @Override
+ public void channelInactive(ChannelContext ctx) {
+
+ }
+
+ @Override
+ public void exceptionCaught(Throwable cause) {
+
+ }
+
+ @Override
+ public void flush(ChannelContext ctx) {
+ ctx.flush();
+ }
+
+ @Override
+ public void setSource(ChannelContext ctx, Object source) {
+
+ }
+
+ public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() {
+ @Override
+ public void dataReceived(ChannelContext ctx, Values data) {
+ ctx.emit(data);
+ }
+ };
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
new file mode 100644
index 0000000..6a853be
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+import java.util.Map;
+
+/**
+ * Subclass of AbstractTupleProcessor provides a series of tuple. It
+ * takes a series of iterators of {@link Values} and produces a stream of
+ * tuple.
+ *
+ * The subclass implements the {@see next()} method to provide
+ * the output of the stream. It can choose to return null in {@see next()} to
+ * indicate that this particular iteration is a no-op. SQL processors depend
+ * on this semantic to implement filtering and nullable records.
+ */
+public abstract class AbstractValuesProcessor {
+
+ /**
+ * Initialize the data sources.
+ *
+ * @param data a map from the table name to the iterators of the values.
+ *
+ */
+ public abstract void initialize(Map<String, DataSource> data, ChannelHandler
+ result);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
new file mode 100644
index 0000000..65ad01c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public interface ChannelContext {
+ /**
+ * Emit data to the next stage of the data pipeline.
+ */
+ void emit(Values data);
+ void fireChannelInactive();
+ void flush();
+ void setSource(Object source);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
new file mode 100644
index 0000000..af02b7e
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+/**
+ * DataListener provides an event-driven interface for the user to process
+ * series of events.
+ */
+public interface ChannelHandler {
+ void dataReceived(ChannelContext ctx, Values data);
+
+ /**
+ * The producer of the data has indicated that the channel is no longer
+ * active.
+ * @param ctx
+ */
+ void channelInactive(ChannelContext ctx);
+
+ void exceptionCaught(Throwable cause);
+
+ void flush(ChannelContext ctx);
+
+ void setSource(ChannelContext ctx, Object source);
+}
[08/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
new file mode 100644
index 0000000..975885b
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
@@ -0,0 +1,24 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopology"
+ constructorArgs:
+ - "foo"
+ - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
new file mode 100644
index 0000000..978181b
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
@@ -0,0 +1,24 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+name: "existing-topology"
+
+config:
+ topology.workers: 1
+
+topologySource:
+ className: "org.apache.storm.flux.test.TridentTopologySource"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology.yaml b/flux/flux-core/src/test/resources/configs/existing-topology.yaml
new file mode 100644
index 0000000..e112c0f
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology.yaml
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopologySource"
+ constructorArgs:
+ - "foo"
+ - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/hdfs_test.yaml b/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
new file mode 100644
index 0000000..2bccb33
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
@@ -0,0 +1,97 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hdfs-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+ - id: "syncPolicy"
+ className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
+ constructorArgs:
+ - 1000
+ - id: "rotationPolicy"
+ className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
+ constructorArgs:
+ - 5.0
+ - MB
+
+ - id: "fileNameFormat"
+ className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
+ configMethods:
+ - name: "withPath"
+ args: ["/tmp/foo/"]
+ - name: "withExtension"
+ args: [".txt"]
+
+ - id: "recordFormat"
+ className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
+ configMethods:
+ - name: "withFieldDelimiter"
+ args: ["|"]
+
+ - id: "rotationAction"
+ className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
+ configMethods:
+ - name: "toDestination"
+ args: ["/tmp/dest2"]
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+ # ...
+
+# bolt definitions
+
+# HdfsBolt bolt = new HdfsBolt()
+# .withConfigKey("hdfs.config")
+# .withFsUrl(args[0])
+# .withFileNameFormat(fileNameFormat)
+# .withRecordFormat(format)
+# .withRotationPolicy(rotationPolicy)
+# .withSyncPolicy(syncPolicy)
+# .addRotationAction(new MoveFileAction().toDestination("/tmp/dest2/"));
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.hdfs.bolt.HdfsBolt"
+ configMethods:
+ - name: "withConfigKey"
+ args: ["hdfs.config"]
+ - name: "withFsUrl"
+ args: ["hdfs://hadoop:54310"]
+ - name: "withFileNameFormat"
+ args: [ref: "fileNameFormat"]
+ - name: "withRecordFormat"
+ args: [ref: "recordFormat"]
+ - name: "withRotationPolicy"
+ args: [ref: "rotationPolicy"]
+ - name: "withSyncPolicy"
+ args: [ref: "syncPolicy"]
+ - name: "addRotationAction"
+ args: [ref: "rotationAction"]
+ parallelism: 1
+ # ...
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/include_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/include_test.yaml b/flux/flux-core/src/test/resources/configs/include_test.yaml
new file mode 100644
index 0000000..702f590
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/include_test.yaml
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test includes by defining nothing, and simply override the topology name
+---
+
+name: "include-topology"
+
+includes:
+ - resource: true
+ file: "/configs/shell_test.yaml"
+ override: false #otherwise subsequent includes that define 'name' would override
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml b/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
new file mode 100644
index 0000000..c2dfac0
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
@@ -0,0 +1,33 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# This is an invalid config. It defines both a topologySource and a list of spouts.
+---
+
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopologySource"
+
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ parallelism: 1
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/kafka_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/kafka_test.yaml b/flux/flux-core/src/test/resources/configs/kafka_test.yaml
new file mode 100644
index 0000000..1fb59ca
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/kafka_test.yaml
@@ -0,0 +1,126 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+# topology definition
+# name to be used when submitting
+name: "kafka-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+ - id: "stringScheme"
+ className: "org.apache.storm.kafka.StringScheme"
+
+ - id: "stringMultiScheme"
+ className: "org.apache.storm.spout.SchemeAsMultiScheme"
+ constructorArgs:
+ - ref: "stringScheme"
+
+ - id: "zkHosts"
+ className: "org.apache.storm.kafka.ZkHosts"
+ constructorArgs:
+ - "localhost:2181"
+
+# Alternative kafka config
+# - id: "kafkaConfig"
+# className: "org.apache.storm.kafka.KafkaConfig"
+# constructorArgs:
+# # brokerHosts
+# - ref: "zkHosts"
+# # topic
+# - "myKafkaTopic"
+# # clientId (optional)
+# - "myKafkaClientId"
+
+ - id: "spoutConfig"
+ className: "org.apache.storm.kafka.SpoutConfig"
+ constructorArgs:
+ # brokerHosts
+ - ref: "zkHosts"
+ # topic
+ - "myKafkaTopic"
+ # zkRoot
+ - "/kafkaSpout"
+ # id
+ - "myId"
+ properties:
+ - name: "ignoreZkOffsets"
+ value: true
+ - name: "scheme"
+ ref: "stringMultiScheme"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "kafka-spout"
+ className: "org.apache.storm.kafka.KafkaSpout"
+ constructorArgs:
+ - ref: "spoutConfig"
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "kafka-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/shell_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/shell_test.yaml b/flux/flux-core/src/test/resources/configs/shell_test.yaml
new file mode 100644
index 0000000..dfab397
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/shell_test.yaml
@@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+# - id: "myComponent"
+# className: "com.foo.bar.MyComponent"
+# constructorArgs:
+# - ...
+# properties:
+# foo: "bar"
+# bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ configMethods:
+ - name: "addComponentConfig"
+ args: ["rabbitmq.configfile", "etc/rabbit.yml"]
+ - name: "addComponentConfig"
+ args:
+ - "publisher.data_paths"
+ - ["actions"]
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ configMethods:
+ - name: "addComponentConfig"
+ args: ["rabbitmq.configfile", "etc/rabbit.yml"]
+ - name: "addComponentConfig"
+ args:
+ - "publisher.data_paths"
+ - ["actions"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "sentence-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/simple_hbase.yaml b/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
new file mode 100644
index 0000000..b841b53
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
@@ -0,0 +1,120 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hbase-wordcount"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+
+# WordSpout spout = new WordSpout();
+# WordCounter bolt = new WordCounter();
+#
+# SimpleHBaseMapper mapper = new SimpleHBaseMapper()
+# .withRowKeyField("word")
+# .withColumnFields(new Fields("word"))
+# .withCounterFields(new Fields("count"))
+# .withColumnFamily("cf");
+#
+# HBaseBolt hbase = new HBaseBolt("WordCount", mapper)
+# .withConfigKey("hbase.conf");
+#
+#
+# // wordSpout ==> countBolt ==> HBaseBolt
+# TopologyBuilder builder = new TopologyBuilder();
+#
+# builder.setSpout(WORD_SPOUT, spout, 1);
+# builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
+# builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
+
+
+
+
+components:
+ - id: "columnFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ - ["word"]
+
+ - id: "counterFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ - ["count"]
+
+ - id: "mapper"
+ className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+ configMethods:
+ - name: "withRowKeyField"
+ args: ["word"]
+ - name: "withColumnFields"
+ args: [ref: "columnFields"]
+ - name: "withCounterFields"
+ args: [ref: "counterFields"]
+ - name: "withColumnFamily"
+ args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ hbase.conf:
+ hbase.rootdir: "hdfs://hadoop:54310/hbase"
+ hbase.zookeeper.quorum: "hadoop"
+
+# spout definitions
+spouts:
+ - id: "word-spout"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+
+bolts:
+ - id: "count-bolt"
+ className: "org.apache.storm.testing.TestWordCounter"
+
+ - id: "hbase-bolt"
+ className: "org.apache.storm.hbase.bolt.HBaseBolt"
+ constructorArgs:
+ - "WordCount" # HBase table name
+ - ref: "mapper"
+ configMethods:
+ - name: "withConfigKey"
+ args: ["hbase.conf"]
+ parallelism: 1
+
+
+streams:
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "word-spout"
+ to: "count-bolt"
+ grouping:
+ type: SHUFFLE
+
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "count-bolt"
+ to: "hbase-bolt"
+ grouping:
+ type: FIELDS
+ args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/substitution-test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/substitution-test.yaml b/flux/flux-core/src/test/resources/configs/substitution-test.yaml
new file mode 100644
index 0000000..ce9e62d
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/substitution-test.yaml
@@ -0,0 +1,106 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "${topology.name}"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+# - id: "myComponent"
+# className: "com.foo.bar.MyComponent"
+# constructorArgs:
+# - ...
+# properties:
+# foo: "bar"
+# bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # test environent variable substitution
+ test.env.value: ${ENV-PATH}
+ # ...
+
+# spout definitions
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "sentence-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/tck.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/tck.yaml b/flux/flux-core/src/test/resources/configs/tck.yaml
new file mode 100644
index 0000000..5d40445
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/tck.yaml
@@ -0,0 +1,95 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+# YAML configuration to serve as a basic smoke test for what is supported.
+#
+# We should support comments, so if we've failed so far, things aren't good.
+
+# we shouldn't choke if we see a document separator...
+---
+
+# topology definition
+# name to be used when submitting
+name: "yaml-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+# - id: "myComponent"
+# className: "com.foo.bar.MyComponent"
+# properties:
+# foo: "bar"
+# bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "bolt-1 --> bolt2"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: CUSTOM
+ customClass:
+ className: "org.apache.storm.testing.NGrouping"
+ constructorArgs:
+ - 1
+
+
+
+
+
+
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/test.properties
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/test.properties b/flux/flux-core/src/test/resources/configs/test.properties
new file mode 100644
index 0000000..ecd89d9
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/test.properties
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+topology.name: substitution-topology
+some.other.property: foo bar
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/log4j2.xml b/flux/flux-core/src/test/resources/log4j2.xml
new file mode 100644
index 0000000..d3333f3
--- /dev/null
+++ b/flux/flux-core/src/test/resources/log4j2.xml
@@ -0,0 +1,34 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<Configuration scan="true" monitorInterval="30">
+ <Appenders>
+ <Console name="A1" target="SYSTEM_OUT">
+ <PatternLayout>
+ <pattern>%-4r [%t] %-5level %c{1} - %msg%n</pattern>
+ </PatternLayout>
+ </Console>
+ </Appenders>
+ <Loggers>
+ <Logger name="org.apache.storm.curator" level="warn"/>
+ <Logger name="org.apache.storm.flux" level="debug"/>
+ <Logger name="org.apache.storm.zookeeper" level="warn"/>
+ <Root level="debug">
+ <appender-ref ref="A1"/>
+ </Root>
+ </Loggers>
+</Configuration>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/README.md
----------------------------------------------------------------------
diff --git a/flux/flux-examples/README.md b/flux/flux-examples/README.md
new file mode 100644
index 0000000..3d610b4
--- /dev/null
+++ b/flux/flux-examples/README.md
@@ -0,0 +1,93 @@
+# Flux Examples
+A collection of examples illustrating various capabilities.
+
+## Building From Source and Running
+
+Checkout the projects source and perform a top level Maven build (i.e. from the `flux` directory):
+
+```bash
+git clone https://github.com/apache/storm.git
+cd storm
+mvn install -DskipTests=true
+```
+
+This will create a shaded (i.e. "fat" or "uber") jar in the `external/flux/flux-examples/target` directory that can run/deployed with
+the `storm` command:
+
+```bash
+cd flux-examples
+storm jar ./target/flux-examples-*-SNAPSHOT.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_wordcount.yaml
+```
+
+The example YAML files are also packaged in the examples jar, so they can also be referenced with Flux's `--resource`
+command line switch:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local --resource /simple_wordcount.yaml
+```
+
+## Available Examples
+
+### [simple_wordcount.yaml](src/main/resources/simple_wordcount.yaml)
+
+This is a very basic wordcount example using Java spouts and bolts. It simply logs the running count of each word
+received.
+
+### [multilang.yaml](src/main/resources/multilang.yaml)
+
+Another wordcount example that uses a spout written in JavaScript (node.js), a bolt written in Python, and two bolts
+written in java.
+
+### [kafka_spout.yaml](src/main/resources/kafka_spout.yaml)
+
+This example illustrates how to configure Storm's `storm-kafka` spout using Flux YAML DSL `components`, `references`,
+and `constructor arguments` constructs.
+
+### [simple_hdfs.yaml](src/main/resources/simple_hdfs.yaml)
+
+This example demonstrates using Flux to setup a storm-hdfs bolt to write to an HDFS cluster. It also demonstrates Flux's
+variable substitution/filtering feature.
+
+To run the `simple_hdfs.yaml` example, copy the `hdfs_bolt.properties` file to a convenient location and change, at
+least, the property `hdfs.url` to point to a HDFS cluster. Then you can run the example something like:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hdfs.yaml --filter my_hdfs_bolt.properties
+```
+
+### [simple_hbase.yaml](src/main/resources/simple_hbase.yaml)
+
+This example illustrates how to use Flux to setup a storm-hbase bolt to write to HBase.
+
+To run the `simple_hbase.yaml` example, copy the `hbase_bolt.properties` file to a convenient location and change the properties
+ `hbase.rootdir` and `hbase.zookeeper.quorum`. Then you can run the example something like:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hbase.yaml --filter my_hbase_bolt.properties
+```
+
+### [simple_windowing.yaml](src/main/resources/simple_windowing.yaml)
+
+This example illustrates how to use Flux to set up a storm topology that contains windowing operations.
+
+To run,
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_windowing.yaml
+```
+
+### [simple_stateful_wordcount.yaml](src/main/resources/simple_stateful_wordcount.yaml)
+
+Flux also supports stateful bolts which is illustrated with this example. It is basically an extension of the basic wordcount example.
+The state is periodically saved (checkpointed) and restored when the topology is restarted.
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
+```
+
+By default the state is stored in-memory only. As such you won't see a resumed state unless you configure to use Redis as the state backend.
+Ensure that you have Redis running at `localhost:6379` and that `storm-redis-*.jar` is in the classpath.
+
+```bash
+STORM_EXT_CLASSPATH=../../storm-redis/target storm jar ./target/flux-examples-*.jar -c topology.state.provider=org.apache.storm.redis.state.RedisKeyValueStateProvider org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
+```
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/pom.xml b/flux/flux-examples/pom.xml
new file mode 100644
index 0000000..ef56ebd
--- /dev/null
+++ b/flux/flux-examples/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>flux-examples</artifactId>
+ <packaging>jar</packaging>
+
+ <name>flux-examples</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-wrappers</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hdfs</artifactId>
+ <version>${project.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>commons-beanutils</groupId>
+ <artifactId>commons-beanutils-core</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>commons-beanutils</groupId>
+ <artifactId>commons-beanutils</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.servlet</groupId>
+ <artifactId>javax.servlet-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hbase</artifactId>
+ <version>${project.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.mortbay.jetty</groupId>
+ <artifactId>servlet-api-2.5</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>commons-collections</groupId>
+ <artifactId>commons-collections</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-kafka</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>${storm.kafka.artifact.id}</artifactId>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <configuration>
+ <createDependencyReducedPom>true</createDependencyReducedPom>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.sf</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.dsa</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ <exclude>META-INF/*.rsa</exclude>
+ <exclude>META-INF/*.EC</exclude>
+ <exclude>META-INF/*.ec</exclude>
+ <exclude>META-INF/MSFTSIG.SF</exclude>
+ <exclude>META-INF/MSFTSIG.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <transformers>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+ <mainClass>org.apache.storm.flux.Flux</mainClass>
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
new file mode 100644
index 0000000..5534888
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.examples;
+
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.base.BaseStatefulBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+
+import java.util.Map;
+
+public class StatefulWordCounter extends BaseStatefulBolt<KeyValueState<String, Long>> {
+
+ private KeyValueState<String, Long> wordCounts;
+ private OutputCollector collector;
+
+ @SuppressWarnings("rawtypes")
+ @Override
+ public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void initState(KeyValueState<String, Long> state) {
+ wordCounts = state;
+ }
+
+ @Override
+ public void execute(Tuple tuple) {
+ String word = tuple.getString(0);
+
+ Long count = wordCounts.get(word, 0L);
+ count++;
+ wordCounts.put(word, count);
+
+ collector.emit(tuple, new Values(word, count));
+ collector.ack(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("word", "count"));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
new file mode 100644
index 0000000..137e354
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.examples;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Prints the tuples to stdout
+ */
+public class TestPrintBolt extends BaseBasicBolt {
+
+ @Override
+ public void execute(Tuple tuple, BasicOutputCollector collector) {
+ System.out.println(tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer ofd) {
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
new file mode 100644
index 0000000..8c904d9
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.examples;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.windowing.TupleWindow;
+
+import java.util.Map;
+
+public class TestWindowBolt extends BaseWindowedBolt {
+ private OutputCollector collector;
+
+ @Override
+ public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+ this.collector = collector;
+ }
+
+ @Override
+ public void execute(TupleWindow inputWindow) {
+ collector.emit(new Values(inputWindow.get().size()));
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("count"));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
new file mode 100644
index 0000000..eb4fb7a
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.examples;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.FileInputStream;
+import java.util.Properties;
+
+/**
+ * Connects to the 'WordCount' HBase table and prints counts for each word.
+ *
+ * Assumes you have run (or are running) the YAML topology definition in
+ * <code>simple_hbase.yaml</code>
+ *
+ * You will also need to modify `src/main/resources/hbase-site.xml`
+ * to point to your HBase instance, and then repackage with `mvn package`.
+ * This is a known issue.
+ *
+ */
+public class WordCountClient {
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = HBaseConfiguration.create();
+ if(args.length == 1){
+ Properties props = new Properties();
+ props.load(new FileInputStream(args[0]));
+ System.out.println("HBase configuration:");
+ for(Object key : props.keySet()) {
+ System.out.println(key + "=" + props.get(key));
+ config.set((String)key, props.getProperty((String)key));
+ }
+ } else {
+ System.out.println("Usage: WordCountClient <hbase_config.properties>");
+ System.exit(1);
+ }
+
+ HTable table = new HTable(config, "WordCount");
+ String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
+
+ for (String word : words) {
+ Get get = new Get(Bytes.toBytes(word));
+ Result result = table.get(get);
+
+ byte[] countBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("count"));
+ byte[] wordBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("word"));
+
+ String wordStr = Bytes.toString(wordBytes);
+ long count = Bytes.toLong(countBytes);
+ System.out.println("Word: '" + wordStr + "', Count: " + count);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
new file mode 100644
index 0000000..7093105
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.examples;
+
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+import static org.apache.storm.utils.Utils.tuple;
+
+/**
+ * This bolt is used by the HBase example. It simply emits the first field
+ * found in the incoming tuple as "word", with a "count" of `1`.
+ *
+ * In this case, the downstream HBase bolt handles the counting, so a value
+ * of `1` will just increment the HBase counter by one.
+ */
+public class WordCounter extends BaseBasicBolt {
+ private static final Logger LOG = LoggerFactory.getLogger(WordCounter.class);
+
+
+
+ @SuppressWarnings("rawtypes")
+ public void prepare(Map stormConf, TopologyContext context) {
+ }
+
+ /*
+ * Just output the word value with a count of 1.
+ * The HBaseBolt will handle incrementing the counter.
+ */
+ public void execute(Tuple input, BasicOutputCollector collector) {
+ collector.emit(tuple(input.getValues().get(0), 1));
+ }
+
+ public void cleanup() {
+
+ }
+
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ declarer.declare(new Fields("word", "count"));
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return null;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/hbase_bolt.properties
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/hbase_bolt.properties b/flux/flux-examples/src/main/resources/hbase_bolt.properties
new file mode 100644
index 0000000..f8ed50c
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/hbase_bolt.properties
@@ -0,0 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+hbase.rootdir=hdfs://hadoop:54310/hbase
+hbase.zookeeper.quorum=hadoop
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/hdfs_bolt.properties
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/hdfs_bolt.properties b/flux/flux-examples/src/main/resources/hdfs_bolt.properties
new file mode 100644
index 0000000..7bcbe7a
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/hdfs_bolt.properties
@@ -0,0 +1,26 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+# The HDFS url
+hdfs.url=hdfs://hadoop:54310
+
+# The HDFS directory where the bolt will write incoming data
+hdfs.write.dir=/incoming
+
+# The HDFS directory where files will be moved once the bolt has
+# finished writing to it.
+hdfs.dest.dir=/complete
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/kafka_spout.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/kafka_spout.yaml b/flux/flux-examples/src/main/resources/kafka_spout.yaml
new file mode 100644
index 0000000..7533ce4
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/kafka_spout.yaml
@@ -0,0 +1,136 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "kafka-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+ - id: "stringScheme"
+ className: "org.apache.storm.kafka.StringScheme"
+
+ - id: "stringMultiScheme"
+ className: "org.apache.storm.spout.SchemeAsMultiScheme"
+ constructorArgs:
+ - ref: "stringScheme"
+
+ - id: "zkHosts"
+ className: "org.apache.storm.kafka.ZkHosts"
+ constructorArgs:
+ - "localhost:2181"
+
+# Alternative kafka config
+# - id: "kafkaConfig"
+# className: "org.apache.storm.kafka.KafkaConfig"
+# constructorArgs:
+# # brokerHosts
+# - ref: "zkHosts"
+# # topic
+# - "myKafkaTopic"
+# # clientId (optional)
+# - "myKafkaClientId"
+
+ - id: "spoutConfig"
+ className: "org.apache.storm.kafka.SpoutConfig"
+ constructorArgs:
+ # brokerHosts
+ - ref: "zkHosts"
+ # topic
+ - "myKafkaTopic"
+ # zkRoot
+ - "/kafkaSpout"
+ # id
+ - "myId"
+ properties:
+ - name: "ignoreZkOffsets"
+ value: true
+ - name: "scheme"
+ ref: "stringMultiScheme"
+
+
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "kafka-spout"
+ className: "org.apache.storm.kafka.KafkaSpout"
+ constructorArgs:
+ - ref: "spoutConfig"
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "kafka-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/multilang.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/multilang.yaml b/flux/flux-examples/src/main/resources/multilang.yaml
new file mode 100644
index 0000000..aaab5d3
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/multilang.yaml
@@ -0,0 +1,89 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "sentence-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_hbase.yaml b/flux/flux-examples/src/main/resources/simple_hbase.yaml
new file mode 100644
index 0000000..93a2781
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_hbase.yaml
@@ -0,0 +1,92 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+# NOTE: To use this example, you will need to modify `src/main/resources/hbase-site.xml`
+# to point to your HBase instance, and then repackage with `mvn package`.
+# This is a known issue.
+
+# topology definition
+# name to be used when submitting
+name: "hbase-persistent-wordcount"
+
+# Components
+components:
+ - id: "columnFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ - ["word"]
+
+ - id: "counterFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ - ["count"]
+
+ - id: "mapper"
+ className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+ configMethods:
+ - name: "withRowKeyField"
+ args: ["word"]
+ - name: "withColumnFields"
+ args: [ref: "columnFields"]
+ - name: "withCounterFields"
+ args: [ref: "counterFields"]
+ - name: "withColumnFamily"
+ args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+config:
+ topology.workers: 1
+ hbase.conf:
+ hbase.rootdir: "${hbase.rootdir}"
+ hbase.zookeeper.quorum: "${hbase.zookeeper.quorum}"
+
+# spout definitions
+spouts:
+ - id: "word-spout"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+
+bolts:
+ - id: "count-bolt"
+ className: "org.apache.storm.flux.examples.WordCounter"
+ parallelism: 1
+
+ - id: "hbase-bolt"
+ className: "org.apache.storm.hbase.bolt.HBaseBolt"
+ constructorArgs:
+ - "WordCount" # HBase table name
+ - ref: "mapper"
+ configMethods:
+ - name: "withConfigKey"
+ args: ["hbase.conf"]
+ parallelism: 1
+
+streams:
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "word-spout"
+ to: "count-bolt"
+ grouping:
+ type: SHUFFLE
+
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "count-bolt"
+ to: "hbase-bolt"
+ grouping:
+ type: FIELDS
+ args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_hdfs.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_hdfs.yaml b/flux/flux-examples/src/main/resources/simple_hdfs.yaml
new file mode 100644
index 0000000..b8d4020
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_hdfs.yaml
@@ -0,0 +1,105 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hdfs-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+ - id: "syncPolicy"
+ className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
+ constructorArgs:
+ - 1000
+ - id: "rotationPolicy"
+ className: "org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy"
+ constructorArgs:
+ - 30
+ - SECONDS
+
+ - id: "fileNameFormat"
+ className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
+ configMethods:
+ - name: "withPath"
+ args: ["${hdfs.write.dir}"]
+ - name: "withExtension"
+ args: [".txt"]
+
+ - id: "recordFormat"
+ className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
+ configMethods:
+ - name: "withFieldDelimiter"
+ args: ["|"]
+
+ - id: "rotationAction"
+ className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
+ configMethods:
+ - name: "toDestination"
+ args: ["${hdfs.dest.dir}"]
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+ # ...
+
+# bolt definitions
+
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.hdfs.bolt.HdfsBolt"
+ configMethods:
+ - name: "withConfigKey"
+ args: ["hdfs.config"]
+ - name: "withFsUrl"
+ args: ["${hdfs.url}"]
+ - name: "withFileNameFormat"
+ args: [ref: "fileNameFormat"]
+ - name: "withRecordFormat"
+ args: [ref: "recordFormat"]
+ - name: "withRotationPolicy"
+ args: [ref: "rotationPolicy"]
+ - name: "withSyncPolicy"
+ args: [ref: "syncPolicy"]
+ - name: "addRotationAction"
+ args: [ref: "rotationAction"]
+ parallelism: 1
+ # ...
+
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+streams:
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: SHUFFLE
+
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "spout-1"
+ to: "bolt-2"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml b/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
new file mode 100644
index 0000000..14b9b3a
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "stateful-wordcount-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.flux.examples.StatefulWordCounter"
+ parallelism: 1
+
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "bolt-1 --> bolt2"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_windowing.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_windowing.yaml b/flux/flux-examples/src/main/resources/simple_windowing.yaml
new file mode 100755
index 0000000..f2f74ff
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_windowing.yaml
@@ -0,0 +1,69 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+---
+
+name: "sliding-window-topology"
+
+components:
+ - id: "windowLength"
+ className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
+ constructorArgs:
+ - 5
+ - id: "slidingInterval"
+ className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
+ constructorArgs:
+ - 3
+
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.flux.examples.TestWindowBolt"
+ configMethods:
+ - name: "withWindow"
+ args: [ref: "windowLength", ref: "slidingInterval"]
+ parallelism: 1
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.examples.TestPrintBolt"
+ parallelism: 1
+
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+ - name: "bolt-1 --> bolt-2" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_wordcount.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_wordcount.yaml b/flux/flux-examples/src/main/resources/simple_wordcount.yaml
new file mode 100644
index 0000000..6443a97
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_wordcount.yaml
@@ -0,0 +1,68 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "yaml-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "bolt-1 --> bolt2"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: SHUFFLE
+
+
+
+
+
+
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-ui/README.md
----------------------------------------------------------------------
diff --git a/flux/flux-ui/README.md b/flux/flux-ui/README.md
new file mode 100644
index 0000000..8b6bd5f
--- /dev/null
+++ b/flux/flux-ui/README.md
@@ -0,0 +1,3 @@
+# Flux-UI
+
+Placeholder for Flux GUI
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/pom.xml b/flux/flux-wrappers/pom.xml
new file mode 100644
index 0000000..7d97a9f
--- /dev/null
+++ b/flux/flux-wrappers/pom.xml
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>flux-wrappers</artifactId>
+ <packaging>jar</packaging>
+
+ <name>flux-wrappers</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>multilang-javascript</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>multilang-ruby</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>multilang-python</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ </dependencies>
+
+</project>
[18/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/pom.xml b/external/flux/flux-examples/pom.xml
deleted file mode 100644
index ef56ebd..0000000
--- a/external/flux/flux-examples/pom.xml
+++ /dev/null
@@ -1,146 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux</artifactId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../pom.xml</relativePath>
- </parent>
-
- <artifactId>flux-examples</artifactId>
- <packaging>jar</packaging>
-
- <name>flux-examples</name>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-core</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-wrappers</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-hdfs</artifactId>
- <version>${project.version}</version>
- <exclusions>
- <exclusion>
- <groupId>commons-beanutils</groupId>
- <artifactId>commons-beanutils-core</artifactId>
- </exclusion>
- <exclusion>
- <groupId>commons-beanutils</groupId>
- <artifactId>commons-beanutils</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>javax.servlet</groupId>
- <artifactId>javax.servlet-api</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-hbase</artifactId>
- <version>${project.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.mortbay.jetty</groupId>
- <artifactId>servlet-api-2.5</artifactId>
- </exclusion>
- <exclusion>
- <groupId>commons-collections</groupId>
- <artifactId>commons-collections</artifactId>
- </exclusion>
- <exclusion>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-kafka</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>${storm.kafka.artifact.id}</artifactId>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <configuration>
- <createDependencyReducedPom>true</createDependencyReducedPom>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.sf</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.dsa</exclude>
- <exclude>META-INF/*.RSA</exclude>
- <exclude>META-INF/*.rsa</exclude>
- <exclude>META-INF/*.EC</exclude>
- <exclude>META-INF/*.ec</exclude>
- <exclude>META-INF/MSFTSIG.SF</exclude>
- <exclude>META-INF/MSFTSIG.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
- <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- <mainClass>org.apache.storm.flux.Flux</mainClass>
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
deleted file mode 100644
index 5534888..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.examples;
-
-import org.apache.storm.state.KeyValueState;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.base.BaseStatefulBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-
-import java.util.Map;
-
-public class StatefulWordCounter extends BaseStatefulBolt<KeyValueState<String, Long>> {
-
- private KeyValueState<String, Long> wordCounts;
- private OutputCollector collector;
-
- @SuppressWarnings("rawtypes")
- @Override
- public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void initState(KeyValueState<String, Long> state) {
- wordCounts = state;
- }
-
- @Override
- public void execute(Tuple tuple) {
- String word = tuple.getString(0);
-
- Long count = wordCounts.get(word, 0L);
- count++;
- wordCounts.put(word, count);
-
- collector.emit(tuple, new Values(word, count));
- collector.ack(tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields("word", "count"));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
deleted file mode 100644
index 137e354..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.examples;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-
-/**
- * Prints the tuples to stdout
- */
-public class TestPrintBolt extends BaseBasicBolt {
-
- @Override
- public void execute(Tuple tuple, BasicOutputCollector collector) {
- System.out.println(tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer ofd) {
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
deleted file mode 100644
index 8c904d9..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.examples;
-
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.windowing.TupleWindow;
-
-import java.util.Map;
-
-public class TestWindowBolt extends BaseWindowedBolt {
- private OutputCollector collector;
-
- @Override
- public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
- this.collector = collector;
- }
-
- @Override
- public void execute(TupleWindow inputWindow) {
- collector.emit(new Values(inputWindow.get().size()));
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields("count"));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
deleted file mode 100644
index eb4fb7a..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.examples;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import java.io.FileInputStream;
-import java.util.Properties;
-
-/**
- * Connects to the 'WordCount' HBase table and prints counts for each word.
- *
- * Assumes you have run (or are running) the YAML topology definition in
- * <code>simple_hbase.yaml</code>
- *
- * You will also need to modify `src/main/resources/hbase-site.xml`
- * to point to your HBase instance, and then repackage with `mvn package`.
- * This is a known issue.
- *
- */
-public class WordCountClient {
-
- public static void main(String[] args) throws Exception {
- Configuration config = HBaseConfiguration.create();
- if(args.length == 1){
- Properties props = new Properties();
- props.load(new FileInputStream(args[0]));
- System.out.println("HBase configuration:");
- for(Object key : props.keySet()) {
- System.out.println(key + "=" + props.get(key));
- config.set((String)key, props.getProperty((String)key));
- }
- } else {
- System.out.println("Usage: WordCountClient <hbase_config.properties>");
- System.exit(1);
- }
-
- HTable table = new HTable(config, "WordCount");
- String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
-
- for (String word : words) {
- Get get = new Get(Bytes.toBytes(word));
- Result result = table.get(get);
-
- byte[] countBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("count"));
- byte[] wordBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("word"));
-
- String wordStr = Bytes.toString(wordBytes);
- long count = Bytes.toLong(countBytes);
- System.out.println("Word: '" + wordStr + "', Count: " + count);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
deleted file mode 100644
index 7093105..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.examples;
-
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.IBasicBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-import static org.apache.storm.utils.Utils.tuple;
-
-/**
- * This bolt is used by the HBase example. It simply emits the first field
- * found in the incoming tuple as "word", with a "count" of `1`.
- *
- * In this case, the downstream HBase bolt handles the counting, so a value
- * of `1` will just increment the HBase counter by one.
- */
-public class WordCounter extends BaseBasicBolt {
- private static final Logger LOG = LoggerFactory.getLogger(WordCounter.class);
-
-
-
- @SuppressWarnings("rawtypes")
- public void prepare(Map stormConf, TopologyContext context) {
- }
-
- /*
- * Just output the word value with a count of 1.
- * The HBaseBolt will handle incrementing the counter.
- */
- public void execute(Tuple input, BasicOutputCollector collector) {
- collector.emit(tuple(input.getValues().get(0), 1));
- }
-
- public void cleanup() {
-
- }
-
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- declarer.declare(new Fields("word", "count"));
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return null;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/hbase_bolt.properties b/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
deleted file mode 100644
index f8ed50c..0000000
--- a/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-hbase.rootdir=hdfs://hadoop:54310/hbase
-hbase.zookeeper.quorum=hadoop
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties b/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
deleted file mode 100644
index 7bcbe7a..0000000
--- a/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
+++ /dev/null
@@ -1,26 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-
-# The HDFS url
-hdfs.url=hdfs://hadoop:54310
-
-# The HDFS directory where the bolt will write incoming data
-hdfs.write.dir=/incoming
-
-# The HDFS directory where files will be moved once the bolt has
-# finished writing to it.
-hdfs.dest.dir=/complete
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/kafka_spout.yaml b/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
deleted file mode 100644
index 7533ce4..0000000
--- a/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
+++ /dev/null
@@ -1,136 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "kafka-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
- - id: "stringScheme"
- className: "org.apache.storm.kafka.StringScheme"
-
- - id: "stringMultiScheme"
- className: "org.apache.storm.spout.SchemeAsMultiScheme"
- constructorArgs:
- - ref: "stringScheme"
-
- - id: "zkHosts"
- className: "org.apache.storm.kafka.ZkHosts"
- constructorArgs:
- - "localhost:2181"
-
-# Alternative kafka config
-# - id: "kafkaConfig"
-# className: "org.apache.storm.kafka.KafkaConfig"
-# constructorArgs:
-# # brokerHosts
-# - ref: "zkHosts"
-# # topic
-# - "myKafkaTopic"
-# # clientId (optional)
-# - "myKafkaClientId"
-
- - id: "spoutConfig"
- className: "org.apache.storm.kafka.SpoutConfig"
- constructorArgs:
- # brokerHosts
- - ref: "zkHosts"
- # topic
- - "myKafkaTopic"
- # zkRoot
- - "/kafkaSpout"
- # id
- - "myId"
- properties:
- - name: "ignoreZkOffsets"
- value: true
- - name: "scheme"
- ref: "stringMultiScheme"
-
-
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "kafka-spout"
- className: "org.apache.storm.kafka.KafkaSpout"
- constructorArgs:
- - ref: "spoutConfig"
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "kafka-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/multilang.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/multilang.yaml b/external/flux/flux-examples/src/main/resources/multilang.yaml
deleted file mode 100644
index aaab5d3..0000000
--- a/external/flux/flux-examples/src/main/resources/multilang.yaml
+++ /dev/null
@@ -1,89 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "sentence-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_hbase.yaml b/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
deleted file mode 100644
index 93a2781..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
+++ /dev/null
@@ -1,92 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-# NOTE: To use this example, you will need to modify `src/main/resources/hbase-site.xml`
-# to point to your HBase instance, and then repackage with `mvn package`.
-# This is a known issue.
-
-# topology definition
-# name to be used when submitting
-name: "hbase-persistent-wordcount"
-
-# Components
-components:
- - id: "columnFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- - ["word"]
-
- - id: "counterFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- - ["count"]
-
- - id: "mapper"
- className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
- configMethods:
- - name: "withRowKeyField"
- args: ["word"]
- - name: "withColumnFields"
- args: [ref: "columnFields"]
- - name: "withCounterFields"
- args: [ref: "counterFields"]
- - name: "withColumnFamily"
- args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-config:
- topology.workers: 1
- hbase.conf:
- hbase.rootdir: "${hbase.rootdir}"
- hbase.zookeeper.quorum: "${hbase.zookeeper.quorum}"
-
-# spout definitions
-spouts:
- - id: "word-spout"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-
-bolts:
- - id: "count-bolt"
- className: "org.apache.storm.flux.examples.WordCounter"
- parallelism: 1
-
- - id: "hbase-bolt"
- className: "org.apache.storm.hbase.bolt.HBaseBolt"
- constructorArgs:
- - "WordCount" # HBase table name
- - ref: "mapper"
- configMethods:
- - name: "withConfigKey"
- args: ["hbase.conf"]
- parallelism: 1
-
-streams:
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "word-spout"
- to: "count-bolt"
- grouping:
- type: SHUFFLE
-
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "count-bolt"
- to: "hbase-bolt"
- grouping:
- type: FIELDS
- args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml b/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
deleted file mode 100644
index b8d4020..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
+++ /dev/null
@@ -1,105 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hdfs-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
- - id: "syncPolicy"
- className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
- constructorArgs:
- - 1000
- - id: "rotationPolicy"
- className: "org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy"
- constructorArgs:
- - 30
- - SECONDS
-
- - id: "fileNameFormat"
- className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
- configMethods:
- - name: "withPath"
- args: ["${hdfs.write.dir}"]
- - name: "withExtension"
- args: [".txt"]
-
- - id: "recordFormat"
- className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
- configMethods:
- - name: "withFieldDelimiter"
- args: ["|"]
-
- - id: "rotationAction"
- className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
- configMethods:
- - name: "toDestination"
- args: ["${hdfs.dest.dir}"]
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
- # ...
-
-# bolt definitions
-
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.hdfs.bolt.HdfsBolt"
- configMethods:
- - name: "withConfigKey"
- args: ["hdfs.config"]
- - name: "withFsUrl"
- args: ["${hdfs.url}"]
- - name: "withFileNameFormat"
- args: [ref: "fileNameFormat"]
- - name: "withRecordFormat"
- args: [ref: "recordFormat"]
- - name: "withRotationPolicy"
- args: [ref: "rotationPolicy"]
- - name: "withSyncPolicy"
- args: [ref: "syncPolicy"]
- - name: "addRotationAction"
- args: [ref: "rotationAction"]
- parallelism: 1
- # ...
-
- - id: "bolt-2"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
-streams:
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: SHUFFLE
-
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "spout-1"
- to: "bolt-2"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml b/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
deleted file mode 100644
index 14b9b3a..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
+++ /dev/null
@@ -1,60 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "stateful-wordcount-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.flux.examples.StatefulWordCounter"
- parallelism: 1
-
- - id: "bolt-2"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "bolt-1 --> bolt2"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_windowing.yaml b/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
deleted file mode 100755
index f2f74ff..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
+++ /dev/null
@@ -1,69 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
----
-
-name: "sliding-window-topology"
-
-components:
- - id: "windowLength"
- className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
- constructorArgs:
- - 5
- - id: "slidingInterval"
- className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
- constructorArgs:
- - 3
-
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.flux.examples.TestWindowBolt"
- configMethods:
- - name: "withWindow"
- args: [ref: "windowLength", ref: "slidingInterval"]
- parallelism: 1
- - id: "bolt-2"
- className: "org.apache.storm.flux.examples.TestPrintBolt"
- parallelism: 1
-
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: FIELDS
- args: ["word"]
- - name: "bolt-1 --> bolt-2" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml b/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
deleted file mode 100644
index 6443a97..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
+++ /dev/null
@@ -1,68 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "yaml-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
-
- - id: "bolt-2"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "bolt-1 --> bolt2"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: SHUFFLE
-
-
-
-
-
-
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-ui/README.md
----------------------------------------------------------------------
diff --git a/external/flux/flux-ui/README.md b/external/flux/flux-ui/README.md
deleted file mode 100644
index 8b6bd5f..0000000
--- a/external/flux/flux-ui/README.md
+++ /dev/null
@@ -1,3 +0,0 @@
-# Flux-UI
-
-Placeholder for Flux GUI
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/pom.xml b/external/flux/flux-wrappers/pom.xml
deleted file mode 100644
index 7d97a9f..0000000
--- a/external/flux/flux-wrappers/pom.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux</artifactId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../pom.xml</relativePath>
- </parent>
-
- <artifactId>flux-wrappers</artifactId>
- <packaging>jar</packaging>
-
- <name>flux-wrappers</name>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>multilang-javascript</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>multilang-ruby</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>multilang-python</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
-
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
deleted file mode 100644
index 05b8e7a..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.wrappers.bolts;
-
-import org.apache.storm.task.ShellBolt;
-import org.apache.storm.topology.IRichBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A generic `ShellBolt` implementation that allows you specify output fields
- * and even streams without having to subclass `ShellBolt` to do so.
- *
- */
-public class FluxShellBolt extends ShellBolt implements IRichBolt{
- private Map<String, String[]> outputFields;
- private Map<String, Object> componentConfig;
-
- /**
- * Create a ShellBolt with command line arguments
- * @param command Command line arguments for the bolt
- */
- public FluxShellBolt(String[] command){
- super(command);
- this.outputFields = new HashMap<String, String[]>();
- }
-
- /**
- * Create a ShellBolt with command line arguments and output fields
- *
- * Keep this constructor for backward compatibility.
- *
- * @param command Command line arguments for the bolt
- * @param outputFields Names of fields the bolt will emit (if any).
- */
- public FluxShellBolt(String[] command, String[] outputFields){
- this(command);
- this.setDefaultStream(outputFields);
- }
-
- /**
- * Add configuration for this bolt. This method is called from YAML file:
- *
- * ```
- * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- * constructorArgs:
- * # command line
- * - ["python", "splitsentence.py"]
- * # output fields
- * - ["word"]
- * configMethods:
- * - name: "addComponentConfig"
- * args: ["publisher.data_paths", "actions"]
- * ```
- *
- * @param key
- * @param value
- */
- public void addComponentConfig(String key, Object value) {
- if (this.componentConfig == null) {
- this.componentConfig = new HashMap<String, Object>();
- }
- this.componentConfig.put(key, value);
- }
-
- /**
- * Add configuration for this bolt. This method is called from YAML file:
- *
- * ```
- * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- * constructorArgs:
- * # command line
- * - ["python", "splitsentence.py"]
- * # output fields
- * - ["word"]
- * configMethods:
- * - name: "addComponentConfig"
- * args:
- * - "publisher.data_paths"
- * - ["actions"]
- * ```
- *
- * @param key
- * @param values
- */
- public void addComponentConfig(String key, List<Object> values) {
- if (this.componentConfig == null) {
- this.componentConfig = new HashMap<String, Object>();
- }
- this.componentConfig.put(key, values);
- }
-
- /**
- * Set default stream outputFields, this method is called from YAML file:
- *
- * ```
- * bolts:
- * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
- * id: my_bolt
- * constructorArgs:
- * - [python, my_bolt.py]
- * configMethods:
- * - name: setDefaultStream
- * args:
- * - [word, count]
- * ```
- *
- * @param outputFields Names of fields the bolt will emit (if any) in default stream.
- */
- public void setDefaultStream(String[] outputFields) {
- this.setNamedStream("default", outputFields);
- }
-
- /**
- * Set custom *named* stream outputFields, this method is called from YAML file:
- *
- * ```
- * bolts:
- * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
- * id: my_bolt
- * constructorArgs:
- * - [python, my_bolt.py]
- * configMethods:
- * - name: setNamedStream
- * args:
- * - first
- * - [word, count]
- * ```
- * @param name Name of stream the bolt will emit into.
- * @param outputFields Names of fields the bolt will emit in custom *named* stream.
- */
- public void setNamedStream(String name, String[] outputFields) {
- this.outputFields.put(name, outputFields);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- Iterator it = this.outputFields.entrySet().iterator();
- while (it.hasNext()) {
- Map.Entry entryTuple = (Map.Entry)it.next();
- String key = (String)entryTuple.getKey();
- String[] value = (String[])entryTuple.getValue();
- if(key.equals("default")) {
- declarer.declare(new Fields(value));
- } else {
- declarer.declareStream(key, new Fields(value));
- }
- }
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return this.componentConfig;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
deleted file mode 100644
index 5f0e84b..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.flux.wrappers.bolts;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple bolt that does nothing other than LOG.info() every tuple recieveed.
- *
- */
-public class LogInfoBolt extends BaseBasicBolt {
- private static final Logger LOG = LoggerFactory.getLogger(LogInfoBolt.class);
-
- @Override
- public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
- LOG.info("{}", tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
deleted file mode 100644
index 5fd378d..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.wrappers.spouts;
-
-import org.apache.storm.spout.ShellSpout;
-import org.apache.storm.topology.IRichSpout;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Iterator;
-
-/**
- * A generic `ShellSpout` implementation that allows you specify output fields
- * and even streams without having to subclass `ShellSpout` to do so.
- *
- */
-public class FluxShellSpout extends ShellSpout implements IRichSpout {
- private Map<String, String[]> outputFields;
- private Map<String, Object> componentConfig;
-
- /**
- * Create a ShellSpout with command line arguments
- * @param command Command line arguments for the bolt
- */
- public FluxShellSpout(String[] command){
- super(command);
- this.outputFields = new HashMap<String, String[]>();
- }
-
- /**
- * Create a ShellSpout with command line arguments and output fields
- *
- * Keep this constructor for backward compatibility.
- *
- * @param args Command line arguments for the spout
- * @param outputFields Names of fields the spout will emit.
- */
- public FluxShellSpout(String[] args, String[] outputFields){
- this(args);
- this.setDefaultStream(outputFields);
- }
-
- /**
- * Add configuration for this spout. This method is called from YAML file:
- *
- * ```
- * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
- * constructorArgs:
- * # command line
- * - ["python", "splitsentence.py"]
- * # output fields
- * - ["word"]
- * configMethods:
- * - name: "addComponentConfig"
- * args: ["publisher.data_paths", "actions"]
- * ```
- *
- * @param key
- * @param value
- */
- public void addComponentConfig(String key, Object value) {
- if (this.componentConfig == null) {
- this.componentConfig = new HashMap<String, Object>();
- }
- this.componentConfig.put(key, value);
- }
-
- /**
- * Add configuration for this spout. This method is called from YAML file:
- *
- * ```
- * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
- * constructorArgs:
- * # command line
- * - ["python", "splitsentence.py"]
- * # output fields
- * - ["word"]
- * configMethods:
- * - name: "addComponentConfig"
- * args:
- * - "publisher.data_paths"
- * - ["actions"]
- * ```
- *
- * @param key
- * @param values
- */
- public void addComponentConfig(String key, List<Object> values) {
- if (this.componentConfig == null) {
- this.componentConfig = new HashMap<String, Object>();
- }
- this.componentConfig.put(key, values);
- }
-
- /**
- * Set default stream outputFields, this method is called from YAML file:
- *
- * ```
- * spouts:
- * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
- * id: my_spout
- * constructorArgs:
- * - [python, my_spout.py]
- * configMethods:
- * - name: setDefaultStream
- * args:
- * - [word, count]
- * ```
- *
- * @param outputFields Names of fields the spout will emit (if any) in default stream.
- */
- public void setDefaultStream(String[] outputFields) {
- this.setNamedStream("default", outputFields);
- }
-
- /**
- * Set custom *named* stream outputFields, this method is called from YAML file:
- *
- * ```
- * spouts:
- * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
- * id: my_spout
- * constructorArgs:
- * - [python, my_spout.py]
- * configMethods:
- * - name: setNamedStream
- * args:
- * - first
- * - [word, count]
- * ```
- * @param name Name of stream the spout will emit into.
- * @param outputFields Names of fields the spout will emit in custom *named* stream.
- */
- public void setNamedStream(String name, String[] outputFields) {
- this.outputFields.put(name, outputFields);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer declarer) {
- Iterator it = this.outputFields.entrySet().iterator();
- while (it.hasNext()) {
- Map.Entry entryTuple = (Map.Entry)it.next();
- String key = (String)entryTuple.getKey();
- String[] value = (String[])entryTuple.getValue();
- if(key.equals("default")) {
- declarer.declare(new Fields(value));
- } else {
- declarer.declareStream(key, new Fields(value));
- }
- }
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- return this.componentConfig;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js b/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
deleted file mode 100644
index b121915..0000000
--- a/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Example for storm spout. Emits random sentences.
- * The original class in java - org.apache.storm.starter.spout.RandomSentenceSpout.
- *
- */
-
-var storm = require('./storm');
-var Spout = storm.Spout;
-
-
-var SENTENCES = [
- "the cow jumped over the moon",
- "an apple a day keeps the doctor away",
- "four score and seven years ago",
- "snow white and the seven dwarfs",
- "i am at two with nature"]
-
-function RandomSentenceSpout(sentences) {
- Spout.call(this);
- this.runningTupleId = 0;
- this.sentences = sentences;
- this.pending = {};
-};
-
-RandomSentenceSpout.prototype = Object.create(Spout.prototype);
-RandomSentenceSpout.prototype.constructor = RandomSentenceSpout;
-
-RandomSentenceSpout.prototype.getRandomSentence = function() {
- return this.sentences[getRandomInt(0, this.sentences.length - 1)];
-}
-
-RandomSentenceSpout.prototype.nextTuple = function(done) {
- var self = this;
- var sentence = this.getRandomSentence();
- var tup = [sentence];
- var id = this.createNextTupleId();
- this.pending[id] = tup;
- //This timeout can be removed if TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS is configured to 100
- setTimeout(function() {
- self.emit({tuple: tup, id: id}, function(taskIds) {
- self.log(tup + ' sent to task ids - ' + taskIds);
- });
- done();
- },100);
-}
-
-RandomSentenceSpout.prototype.createNextTupleId = function() {
- var id = this.runningTupleId;
- this.runningTupleId++;
- return id;
-}
-
-RandomSentenceSpout.prototype.ack = function(id, done) {
- this.log('Received ack for - ' + id);
- delete this.pending[id];
- done();
-}
-
-RandomSentenceSpout.prototype.fail = function(id, done) {
- var self = this;
- this.log('Received fail for - ' + id + '. Retrying.');
- this.emit({tuple: this.pending[id], id:id}, function(taskIds) {
- self.log(self.pending[id] + ' sent to task ids - ' + taskIds);
- });
- done();
-}
-
-/**
- * Returns a random integer between min (inclusive) and max (inclusive)
- */
-function getRandomInt(min, max) {
- return Math.floor(Math.random() * (max - min + 1)) + min;
-}
-
-new RandomSentenceSpout(SENTENCES).run();
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py b/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
deleted file mode 100644
index 300105f..0000000
--- a/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
+++ /dev/null
@@ -1,24 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-import storm
-
-class SplitSentenceBolt(storm.BasicBolt):
- def process(self, tup):
- words = tup.values[0].split(" ")
- for word in words:
- storm.emit([word])
-
-SplitSentenceBolt().run()
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/pom.xml b/external/flux/pom.xml
deleted file mode 100644
index b51a505..0000000
--- a/external/flux/pom.xml
+++ /dev/null
@@ -1,85 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>flux</artifactId>
- <packaging>pom</packaging>
- <name>flux</name>
-
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../pom.xml</relativePath>
- </parent>
-
- <developers>
- <developer>
- <id>ptgoetz</id>
- <name>P. Taylor Goetz</name>
- <email>ptgoetz@apache.org</email>
- </developer>
- </developers>
-
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- <!-- see comment below... This fixes an annoyance with intellij -->
- <provided.scope>provided</provided.scope>
- </properties>
-
- <profiles>
- <!--
- Hack to make intellij behave.
- If you use intellij, enable this profile in your IDE.
- It should make life easier.
- -->
- <profile>
- <id>intellij</id>
- <properties>
- <provided.scope>compile</provided.scope>
- </properties>
- </profile>
- </profiles>
-
- <modules>
- <module>flux-wrappers</module>
- <module>flux-core</module>
- <module>flux-examples</module>
- </modules>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>${provided.scope}</scope>
- </dependency>
- <dependency>
- <groupId>commons-cli</groupId>
- <artifactId>commons-cli</artifactId>
- <version>1.2</version>
- </dependency>
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>${storm.kafka.artifact.id}</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/README.md
----------------------------------------------------------------------
diff --git a/external/sql/README.md b/external/sql/README.md
deleted file mode 100644
index a4b44fb..0000000
--- a/external/sql/README.md
+++ /dev/null
@@ -1,207 +0,0 @@
-# Storm SQL
-
-Compile SQL queries to Storm topologies.
-
-## Usage
-
-Run the ``storm sql`` command to compile SQL statements into Trident topology, and submit it to the Storm cluster
-
-```
-$ bin/storm sql <sql-file> <topo-name>
-```
-
-In which `sql-file` contains a list of SQL statements to be executed, and `topo-name` is the name of the topology.
-
-StormSQL activates `explain mode` and shows query plan instead of submitting topology when user specifies `topo-name` as `--explain`.
-Detailed explanation is available from `Showing Query Plan (explain mode)` section.
-
-## Supported Features
-
-The following features are supported in the current repository:
-
-* Streaming from and to external data sources
-* Filtering tuples
-* Projections
-* Aggregations (Grouping)
-* User defined function (scalar and aggregate)
-* Join (Inner, Left outer, Right outer, Full outer)
-
-## Specifying External Data Sources
-
-In StormSQL data is represented by external tables. Users can specify data sources using the `CREATE EXTERNAL TABLE`
-statement. For example, the following statement specifies a Kafka spouts and sink:
-
-```
-CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY) LOCATION 'kafka://localhost:2181/brokers?topic=test' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-```
-
-The syntax of `CREATE EXTERNAL TABLE` closely follows the one defined in
-[Hive Data Definition Language](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL).
-
-`PARALLELISM` is StormSQL's own keyword which describes parallelism hint for input data source. This is same as providing parallelism hint to Trident Spout.
-Downstream operators are executed with same parallelism before repartition (Aggregation triggers repartition).
-
-Default value is 1, and this option is no effect on output data source. (We might change if needed. Normally repartition is the thing to avoid.)
-
-## Plugging in External Data Sources
-
-Users plug in external data sources through implementing the `ISqlTridentDataSource` interface and registers them using
-the mechanisms of Java's service loader. The external data source will be chosen based on the scheme of the URI of the
-tables. Please refer to the implementation of `storm-sql-kafka` for more details.
-
-## Specifying User Defined Function (UDF)
-
-Users can define user defined function (scalar or aggregate) using `CREATE FUNCTION` statement.
-For example, the following statement defines `MYPLUS` function which uses `org.apache.storm.sql.TestUtils$MyPlus` class.
-
-```
-CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'
-```
-
-Storm SQL determines whether the function as scalar or aggregate by checking which methods are defined.
-If the class defines `evaluate` method, Storm SQL treats the function as `scalar`,
-and if the class defines `add` method, Storm SQL treats the function as `aggregate`.
-
-Example of class for scalar function is here:
-
-```
- public class MyPlus {
- public static Integer evaluate(Integer x, Integer y) {
- return x + y;
- }
- }
-
-```
-
-and class for aggregate function is here:
-
-```
- public class MyConcat {
- public static String init() {
- return "";
- }
- public static String add(String accumulator, String val) {
- return accumulator + val;
- }
- public static String result(String accumulator) {
- return accumulator;
- }
- }
-```
-
-If users don't define `result` method, result is the last return value of `add` method.
-Users need to define `result` method only when we need to transform accumulated value.
-
-## Example: Filtering Kafka Stream
-
-Let's say there is a Kafka stream that represents the transactions of orders. Each message in the stream contains the id
-of the order, the unit price of the product and the quantity of the orders. The goal is to filter orders where the
-transactions are significant and to insert these orders into another Kafka stream for further analysis.
-
-The user can specify the following SQL statements in the SQL file:
-
-```
-CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-
-CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-
-INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
-```
-
-The first statement defines the table `ORDER` which represents the input stream. The `LOCATION` clause specifies the
-ZkHost (`localhost:2181`), the path of the brokers in ZooKeeper (`/brokers`) and the topic (`orders`).
-The `TBLPROPERTIES` clause specifies the configuration of
-[KafkaProducer](http://kafka.apache.org/documentation.html#newproducerconfigs).
-Current implementation of `storm-sql-kafka` requires specifying both `LOCATION` and `TBLPROPERTIES` clauses even though
-the table is read-only or write-only.
-
-Similarly, the second statement specifies the table `LARGE_ORDERS` which represents the output stream. The third
-statement is a `SELECT` statement which defines the topology: it instructs StormSQL to filter all orders in the external
-table `ORDERS`, calculates the total price and inserts matching records into the Kafka stream specified by
-`LARGE_ORDER`.
-
-To run this example, users need to include the data sources (`storm-sql-kafka` in this case) and its dependency in the
-class path. Dependencies for Storm SQL are automatically handled when users run `storm sql`. Users can include data sources at the submission step like below:
-
-```
-$ bin/storm sql order_filtering.sql order_filtering --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
-```
-
-Above command submits the SQL statements to StormSQL. Users need to modify each artifacts' version if users are using different version of Storm or Kafka.
-
-By now you should be able to see the `order_filtering` topology in the Storm UI.
-
-## Showing Query Plan (explain mode)
-
-Like `explain` on SQL statement, StormSQL provides `explain mode` when running Storm SQL Runner. In explain mode, StormSQL analyzes each query statement (only DML) and show plan instead of submitting topology.
-
-In order to run `explain mode`, you need to provide topology name as `--explain` and run `storm sql` as same as submitting.
-
-For example, when you run the example seen above with explain mode:
-
-```
-$ bin/storm sql order_filtering.sql --explain --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
-```
-
-StormSQL prints out like below:
-
-```
-
-===========================================================
-query>
-CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
------------------------------------------------------------
-16:53:43.951 [main] INFO o.a.s.s.r.DataSourcesRegistry - Registering scheme kafka with org.apache.storm.sql.kafka.KafkaDataSourcesProvider@4d1bf319
-No plan presented on DDL
-===========================================================
-===========================================================
-query>
-CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
------------------------------------------------------------
-No plan presented on DDL
-===========================================================
-===========================================================
-query>
-INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
------------------------------------------------------------
-plan>
-LogicalTableModify(table=[[LARGE_ORDERS]], operation=[INSERT], updateColumnList=[[]], flattened=[true]), id = 8
- LogicalProject(ID=[$0], TOTAL=[*($1, $2)]), id = 7
- LogicalFilter(condition=[>(*($1, $2), 50)]), id = 6
- EnumerableTableScan(table=[[ORDERS]]), id = 5
-
-===========================================================
-
-```
-
-## Current Limitations
-
-- Windowing is yet to be implemented.
-- Only equi-join (single field equality) is supported for joining table.
-- Joining table only applies within each small batch that comes off of the spout.
- - Not across batches.
- - Limitation came from `join` feature of Trident.
- - Please refer this doc: `Trident API Overview` for details.
-
-## License
-
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements. See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership. The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied. See the License for the
-specific language governing permissions and limitations
-under the License.
-
-## Committer Sponsors
- * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org))
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/pom.xml b/external/sql/pom.xml
deleted file mode 100644
index 02264f7..0000000
--- a/external/sql/pom.xml
+++ /dev/null
@@ -1,47 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../pom.xml</relativePath>
- </parent>
-
- <artifactId>sql</artifactId>
- <packaging>pom</packaging>
-
- <developers>
- <developer>
- <id>haohui</id>
- <name>Haohui Mai</name>
- <email>ricetons@gmail.com</email>
- </developer>
- </developers>
-
- <modules>
- <module>storm-sql-core</module>
- <module>storm-sql-runtime</module>
- <module>storm-sql-external/storm-sql-kafka</module>
- <module>storm-sql-external/storm-sql-redis</module>
- <module>storm-sql-external/storm-sql-mongodb</module>
- <module>storm-sql-external/storm-sql-hdfs</module>
- </modules>
-</project>
[06/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
new file mode 100644
index 0000000..2e237c0
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.*;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.apache.storm.sql.calcite.ParallelStreamableTable;
+import org.apache.storm.sql.parser.ColumnConstraint;
+
+import java.util.ArrayList;
+
+import static org.apache.calcite.rel.RelFieldCollation.Direction;
+import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
+import static org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
+import static org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import static org.apache.calcite.sql.validate.SqlMonotonicity.INCREASING;
+
+public class CompilerUtil {
+ public static String escapeJavaString(String s, boolean nullMeansNull) {
+ if(s == null) {
+ return nullMeansNull ? "null" : "\"\"";
+ } else {
+ String s1 = Util.replace(s, "\\", "\\\\");
+ String s2 = Util.replace(s1, "\"", "\\\"");
+ String s3 = Util.replace(s2, "\n\r", "\\n");
+ String s4 = Util.replace(s3, "\n", "\\n");
+ String s5 = Util.replace(s4, "\r", "\\r");
+ return "\"" + s5 + "\"";
+ }
+ }
+
+ public static class TableBuilderInfo {
+ private final RelDataTypeFactory typeFactory;
+
+ public TableBuilderInfo(RelDataTypeFactory typeFactory) {
+ this.typeFactory = typeFactory;
+ }
+
+ private static class FieldType {
+ private final String name;
+ private final RelDataType relDataType;
+
+ private FieldType(String name, RelDataType relDataType) {
+ this.name = name;
+ this.relDataType = relDataType;
+ }
+
+ }
+
+ private final ArrayList<FieldType> fields = new ArrayList<>();
+ private final ArrayList<Object[]> rows = new ArrayList<>();
+ private int primaryKey = -1;
+ private Integer parallelismHint;
+ private SqlMonotonicity primaryKeyMonotonicity;
+ private Statistic stats;
+
+ public TableBuilderInfo field(String name, SqlTypeName type) {
+ return field(name, typeFactory.createSqlType(type));
+ }
+
+ public TableBuilderInfo field(String name, RelDataType type) {
+ fields.add(new FieldType(name, type));
+ return this;
+ }
+
+ public TableBuilderInfo field(String name, SqlDataTypeSpec type, ColumnConstraint constraint) {
+ RelDataType dataType = type.deriveType(typeFactory);
+ if (constraint instanceof ColumnConstraint.PrimaryKey) {
+ ColumnConstraint.PrimaryKey pk = (ColumnConstraint.PrimaryKey) constraint;
+ Preconditions.checkState(primaryKey == -1, "There are more than one primary key in the table");
+ primaryKey = fields.size();
+ primaryKeyMonotonicity = pk.monotonicity();
+ }
+ fields.add(new FieldType(name, dataType));
+ return this;
+ }
+
+ public TableBuilderInfo statistics(Statistic stats) {
+ this.stats = stats;
+ return this;
+ }
+
+ @VisibleForTesting
+ public TableBuilderInfo rows(Object[] data) {
+ rows.add(data);
+ return this;
+ }
+
+ public TableBuilderInfo parallelismHint(int parallelismHint) {
+ this.parallelismHint = parallelismHint;
+ return this;
+ }
+
+ public StreamableTable build() {
+ final Statistic stat = buildStatistic();
+ final Table tbl = new Table() {
+ @Override
+ public RelDataType getRowType(
+ RelDataTypeFactory relDataTypeFactory) {
+ RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
+ for (FieldType f : fields) {
+ b.add(f.name, f.relDataType);
+ }
+ return b.build();
+ }
+
+ @Override
+ public Statistic getStatistic() {
+ return stat != null ? stat : Statistics.of(rows.size(),
+ ImmutableList.<ImmutableBitSet>of());
+ }
+
+ @Override
+ public Schema.TableType getJdbcTableType() {
+ return Schema.TableType.STREAM;
+ }
+ };
+
+ return new ParallelStreamableTable() {
+ @Override
+ public Integer parallelismHint() {
+ return parallelismHint;
+ }
+
+ @Override
+ public Table stream() {
+ return tbl;
+ }
+
+ @Override
+ public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
+ return tbl.getRowType(relDataTypeFactory);
+ }
+
+ @Override
+ public Statistic getStatistic() {
+ return tbl.getStatistic();
+ }
+
+ @Override
+ public Schema.TableType getJdbcTableType() {
+ return Schema.TableType.STREAM;
+ }
+ };
+ }
+
+ private Statistic buildStatistic() {
+ if (stats != null || primaryKey == -1) {
+ return stats;
+ }
+ Direction dir = primaryKeyMonotonicity == INCREASING ? ASCENDING : DESCENDING;
+ RelFieldCollation collation = new RelFieldCollation(primaryKey, dir, NullDirection.UNSPECIFIED);
+ return Statistics.of(fields.size(), ImmutableList.of(ImmutableBitSet.of(primaryKey)),
+ ImmutableList.of(RelCollations.of(collation)));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
new file mode 100644
index 0000000..5ac95e0
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.List;
+
+/**
+ * Compiles a scalar expression ({@link org.apache.calcite.rex.RexNode}) to Java source code String.
+ *
+ * This code is inspired by JaninoRexCompiler in Calcite, but while it is returning {@link org.apache.calcite.interpreter.Scalar} which is executable,
+ * we need to pass the source code to compile and serialize instance so that it can be executed on worker efficiently.
+ */
+public class RexNodeToJavaCodeCompiler {
+ private final RexBuilder rexBuilder;
+
+ public RexNodeToJavaCodeCompiler(RexBuilder rexBuilder) {
+ this.rexBuilder = rexBuilder;
+ }
+
+ public BlockStatement compileToBlock(List<RexNode> nodes, RelDataType inputRowType) {
+ final RexProgramBuilder programBuilder =
+ new RexProgramBuilder(inputRowType, rexBuilder);
+ for (RexNode node : nodes) {
+ programBuilder.addProject(node, null);
+ }
+
+ return compileToBlock(programBuilder.getProgram());
+ }
+
+ public BlockStatement compileToBlock(final RexProgram program) {
+ final ParameterExpression context_ =
+ Expressions.parameter(Context.class, "context");
+ final ParameterExpression outputValues_ =
+ Expressions.parameter(Object[].class, "outputValues");
+
+ return compileToBlock(program, context_, outputValues_).toBlock();
+ }
+
+ public String compile(List<RexNode> nodes, RelDataType inputRowType, String className) {
+ final RexProgramBuilder programBuilder =
+ new RexProgramBuilder(inputRowType, rexBuilder);
+ for (RexNode node : nodes) {
+ programBuilder.addProject(node, null);
+ }
+
+ return compile(programBuilder.getProgram(), className);
+ }
+
+ public String compile(final RexProgram program, String className) {
+ final ParameterExpression context_ =
+ Expressions.parameter(Context.class, "context");
+ final ParameterExpression outputValues_ =
+ Expressions.parameter(Object[].class, "outputValues");
+
+ BlockBuilder builder = compileToBlock(program, context_, outputValues_);
+ return baz(context_, outputValues_, builder.toBlock(), className);
+ }
+
+ private BlockBuilder compileToBlock(final RexProgram program, ParameterExpression context_,
+ ParameterExpression outputValues_) {
+ RelDataType inputRowType = program.getInputRowType();
+ final BlockBuilder builder = new BlockBuilder();
+ final JavaTypeFactoryImpl javaTypeFactory =
+ new JavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
+
+ final RexToLixTranslator.InputGetter inputGetter =
+ new RexToLixTranslator.InputGetterImpl(
+ ImmutableList.of(
+ Pair.<Expression, PhysType>of(
+ Expressions.field(context_,
+ BuiltInMethod.CONTEXT_VALUES.field),
+ PhysTypeImpl.of(javaTypeFactory, inputRowType,
+ JavaRowFormat.ARRAY, false))));
+ final Function1<String, RexToLixTranslator.InputGetter> correlates =
+ new Function1<String, RexToLixTranslator.InputGetter>() {
+ public RexToLixTranslator.InputGetter apply(String a0) {
+ throw new UnsupportedOperationException();
+ }
+ };
+ final Expression root =
+ Expressions.field(context_, BuiltInMethod.CONTEXT_ROOT.field);
+ final List<Expression> list =
+ RexToLixTranslator.translateProjects(program, javaTypeFactory, builder,
+ null, root, inputGetter, correlates);
+ for (int i = 0; i < list.size(); i++) {
+ builder.add(
+ Expressions.statement(
+ Expressions.assign(
+ Expressions.arrayIndex(outputValues_,
+ Expressions.constant(i)),
+ list.get(i))));
+ }
+
+ return builder;
+ }
+
+ /** Given a method that implements {@link ExecutableExpression#execute(Context, Object[])},
+ * adds a bridge method that implements {@link ExecutableExpression#execute(Context)}, and
+ * compiles. */
+ static String baz(ParameterExpression context_,
+ ParameterExpression outputValues_, BlockStatement block, String className) {
+ final List<MemberDeclaration> declarations = Lists.newArrayList();
+
+ // public void execute(Context, Object[] outputValues)
+ declarations.add(
+ Expressions.methodDecl(Modifier.PUBLIC, void.class,
+ StormBuiltInMethod.EXPR_EXECUTE2.method.getName(),
+ ImmutableList.of(context_, outputValues_), block));
+
+ // public Object execute(Context)
+ final BlockBuilder builder = new BlockBuilder();
+ final Expression values_ = builder.append("values",
+ Expressions.newArrayBounds(Object.class, 1,
+ Expressions.constant(1)));
+ builder.add(
+ Expressions.statement(
+ Expressions.call(
+ Expressions.parameter(ExecutableExpression.class, "this"),
+ StormBuiltInMethod.EXPR_EXECUTE2.method, context_, values_)));
+ builder.add(
+ Expressions.return_(null,
+ Expressions.arrayIndex(values_, Expressions.constant(0))));
+ declarations.add(
+ Expressions.methodDecl(Modifier.PUBLIC, Object.class,
+ StormBuiltInMethod.EXPR_EXECUTE1.method.getName(),
+ ImmutableList.of(context_), builder.toBlock()));
+
+ final ClassDeclaration classDeclaration =
+ Expressions.classDecl(Modifier.PUBLIC, className, null,
+ ImmutableList.<Type>of(ExecutableExpression.class), declarations);
+
+ return Expressions.toString(Lists.newArrayList(classDeclaration), "\n", false);
+ }
+
+ enum StormBuiltInMethod {
+ EXPR_EXECUTE1(ExecutableExpression.class, "execute", Context.class),
+ EXPR_EXECUTE2(ExecutableExpression.class, "execute", Context.class, Object[].class);
+
+ public final Method method;
+ public final Constructor constructor;
+ public final Field field;
+
+ public static final ImmutableMap<Method, BuiltInMethod> MAP;
+
+ static {
+ final ImmutableMap.Builder<Method, BuiltInMethod> builder =
+ ImmutableMap.builder();
+ for (BuiltInMethod value : BuiltInMethod.values()) {
+ if (value.method != null) {
+ builder.put(value.method, value);
+ }
+ }
+ MAP = builder.build();
+ }
+
+ private StormBuiltInMethod(Method method, Constructor constructor, Field field) {
+ this.method = method;
+ this.constructor = constructor;
+ this.field = field;
+ }
+
+ /**
+ * Defines a method.
+ */
+ StormBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+ this(Types.lookupMethod(clazz, methodName, argumentTypes), null, null);
+ }
+
+ /**
+ * Defines a constructor.
+ */
+ StormBuiltInMethod(Class clazz, Class... argumentTypes) {
+ this(null, Types.lookupConstructor(clazz, argumentTypes), null);
+ }
+
+ /**
+ * Defines a field.
+ */
+ StormBuiltInMethod(Class clazz, String fieldName, boolean dummy) {
+ this(null, null, Types.lookupField(clazz, fieldName));
+ assert dummy : "dummy value for method overloading must be true";
+ }
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
new file mode 100644
index 0000000..21ca063
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+public class StormSqlTypeFactoryImpl extends JavaTypeFactoryImpl {
+
+ public StormSqlTypeFactoryImpl() {
+ }
+
+ public StormSqlTypeFactoryImpl(RelDataTypeSystem typeSystem) {
+ super(typeSystem);
+ }
+
+ @Override
+ public RelDataType toSql(RelDataType type) {
+ if (type instanceof JavaType) {
+ JavaType javaType = (JavaType) type;
+ SqlTypeName sqlTypeName = JavaToSqlTypeConversionRules.instance().lookup(javaType.getJavaClass());
+ if (sqlTypeName == null) {
+ sqlTypeName = SqlTypeName.ANY;
+ }
+ return createTypeWithNullability(createSqlType(sqlTypeName), type.isNullable());
+ }
+ return super.toSql(type);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
new file mode 100644
index 0000000..9dc4ba8
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
@@ -0,0 +1,238 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.storm.tuple.Values;
+
+import java.lang.reflect.Type;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Built-in implementations for some of the standard aggregation operations.
+ * Aggregations can be implemented as a class with the following methods viz. init, add and result.
+ * The class could contain only static methods, only non-static methods or be generic.
+ */
+public class BuiltinAggregateFunctions {
+ // binds the type information and the class implementing the aggregation
+ public static class TypeClass {
+ public static class GenericType {
+ }
+
+ public final Type ty;
+ public final Class<?> clazz;
+
+ private TypeClass(Type ty, Class<?> clazz) {
+ this.ty = ty;
+ this.clazz = clazz;
+ }
+
+ static TypeClass of(Type ty, Class<?> clazz) {
+ return new TypeClass(ty, clazz);
+ }
+ }
+
+ static final Map<String, List<TypeClass>> TABLE = new HashMap<>();
+
+ public static class ByteSum {
+ public static Byte init() {
+ return 0;
+ }
+
+ public static Byte add(Byte accumulator, Byte val) {
+ return (byte) (accumulator + val);
+ }
+
+ public static Byte result(Byte accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class ShortSum {
+ public static Short init() {
+ return 0;
+ }
+
+ public static Short add(Short accumulator, Short val) {
+ return (short) (accumulator + val);
+ }
+
+ public static Short result(Short accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class IntSum {
+ public static Integer init() {
+ return 0;
+ }
+
+ public static Integer add(Integer accumulator, Integer val) {
+ return accumulator + val;
+ }
+
+ public static Integer result(Integer accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class LongSum {
+ public static Long init() {
+ return 0L;
+ }
+
+ public static Long add(Long accumulator, Long val) {
+ return accumulator + val;
+ }
+
+ public static Long result(Long accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class FloatSum {
+ public static Float init() {
+ return 0.0f;
+ }
+
+ public static Float add(Float accumulator, Float val) {
+ return accumulator + val;
+ }
+
+ public static Float result(Float accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class DoubleSum {
+ public static Double init() {
+ return 0.0;
+ }
+
+ public static Double add(Double accumulator, Double val) {
+ return accumulator + val;
+ }
+
+ public static Double result(Double accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class Max<T extends Comparable<T>> {
+ public T init() {
+ return null;
+ }
+
+ public T add(T accumulator, T val) {
+ return (accumulator == null || accumulator.compareTo(val) < 0) ? val : accumulator;
+ }
+
+ public T result(T accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class Min<T extends Comparable<T>> {
+ public T init() {
+ return null;
+ }
+
+ public T add(T accumulator, T val) {
+ return (accumulator == null || accumulator.compareTo(val) > 0) ? val : accumulator;
+ }
+
+ public T result(T accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static class IntAvg {
+ private int count;
+
+ public Integer init() {
+ return 0;
+ }
+
+ public Integer add(Integer accumulator, Integer val) {
+ ++count;
+ return accumulator + val;
+ }
+
+ public Integer result(Integer accumulator) {
+ Integer result = accumulator / count;
+ count = 0;
+ return result;
+ }
+ }
+
+ public static class DoubleAvg {
+ private int count;
+
+ public Double init() {
+ return 0.0;
+ }
+
+ public Double add(Double accumulator, Double val) {
+ ++count;
+ return accumulator + val;
+ }
+
+ public Double result(Double accumulator) {
+ Double result = accumulator / count;
+ count = 0;
+ return result;
+ }
+ }
+
+ public static class Count {
+ public static Long init() {
+ return 0L;
+ }
+
+ public static Long add(Long accumulator, Values vals) {
+ for (Object val : vals) {
+ if (val == null) {
+ return accumulator;
+ }
+ }
+ return accumulator + 1;
+ }
+
+ public static Long result(Long accumulator) {
+ return accumulator;
+ }
+ }
+
+ static {
+ TABLE.put("SUM", ImmutableList.of(
+ TypeClass.of(float.class, FloatSum.class),
+ TypeClass.of(double.class, DoubleSum.class),
+ TypeClass.of(byte.class, ByteSum.class),
+ TypeClass.of(short.class, ShortSum.class),
+ TypeClass.of(long.class, LongSum.class),
+ TypeClass.of(int.class, IntSum.class)));
+ TABLE.put("AVG", ImmutableList.of(
+ TypeClass.of(double.class, DoubleAvg.class),
+ TypeClass.of(int.class, IntAvg.class)));
+ TABLE.put("COUNT", ImmutableList.of(TypeClass.of(long.class, Count.class)));
+ TABLE.put("MAX", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Max.class)));
+ TABLE.put("MIN", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Min.class)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
new file mode 100644
index 0000000..01546ed
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.storm.sql.compiler.CompilerUtil;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.HashSet;
+import java.util.Set;
+
+public class PlanCompiler {
+ private static final Logger LOG = LoggerFactory.getLogger(PlanCompiler.class);
+
+ private static final Joiner NEW_LINE_JOINER = Joiner.on("\n");
+ private static final String PACKAGE_NAME = "org.apache.storm.sql.generated";
+ private static final String PROLOGUE = NEW_LINE_JOINER.join(
+ "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "",
+ "import java.util.Iterator;", "import java.util.Map;", "import java.util.HashMap;",
+ "import java.util.List;", "import java.util.ArrayList;",
+ "import java.util.LinkedHashMap;",
+ "import org.apache.storm.tuple.Values;",
+ "import org.apache.storm.sql.runtime.AbstractChannelHandler;",
+ "import org.apache.storm.sql.runtime.Channels;",
+ "import org.apache.storm.sql.runtime.ChannelContext;",
+ "import org.apache.storm.sql.runtime.ChannelHandler;",
+ "import org.apache.storm.sql.runtime.DataSource;",
+ "import org.apache.storm.sql.runtime.AbstractValuesProcessor;",
+ "import com.google.common.collect.ArrayListMultimap;",
+ "import com.google.common.collect.Multimap;",
+ "import org.apache.calcite.interpreter.Context;",
+ "import org.apache.calcite.interpreter.StormContext;",
+ "import org.apache.calcite.DataContext;",
+ "import org.apache.storm.sql.runtime.calcite.StormDataContext;",
+ "public final class Processor extends AbstractValuesProcessor {",
+ " public final static DataContext dataContext = new StormDataContext();",
+ "");
+ private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
+ " @Override",
+ " public void initialize(Map<String, DataSource> data,",
+ " ChannelHandler result) {",
+ " ChannelContext r = Channels.chain(Channels.voidContext(), result);",
+ ""
+ );
+
+ private final JavaTypeFactory typeFactory;
+
+ public PlanCompiler(JavaTypeFactory typeFactory) {
+ this.typeFactory = typeFactory;
+ }
+
+ private String generateJavaSource(RelNode root) throws Exception {
+ StringWriter sw = new StringWriter();
+ try (PrintWriter pw = new PrintWriter(sw)) {
+ RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+ printPrologue(pw);
+ compiler.traverse(root);
+ printMain(pw, root);
+ printEpilogue(pw);
+ }
+ return sw.toString();
+ }
+
+ private void printMain(PrintWriter pw, RelNode root) {
+ Set<TableScan> tables = new HashSet<>();
+ pw.print(INITIALIZER_PROLOGUE);
+ chainOperators(pw, root, tables);
+ for (TableScan n : tables) {
+ String escaped = CompilerUtil.escapeJavaString(
+ Joiner.on('.').join(n.getTable().getQualifiedName()), true);
+ String r = NEW_LINE_JOINER.join(
+ " if (!data.containsKey(%1$s))",
+ " throw new RuntimeException(\"Cannot find table \" + %1$s);",
+ " data.get(%1$s).open(CTX_%2$d);",
+ "");
+ pw.print(String.format(r, escaped, n.getId()));
+ }
+ pw.print(" }\n");
+ }
+
+ private void chainOperators(PrintWriter pw, RelNode root, Set<TableScan> tables) {
+ doChainOperators(pw, root, tables, "r");
+ }
+
+ private void doChainOperators(PrintWriter pw, RelNode node, Set<TableScan> tables, String parentCtx) {
+ pw.print(
+ String.format(" ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
+ node.getId(), parentCtx, RelNodeCompiler.getStageName(node)));
+ String currentCtx = String.format("CTX_%d", node.getId());
+ if (node instanceof TableScan) {
+ tables.add((TableScan) node);
+ }
+ for (RelNode i : node.getInputs()) {
+ doChainOperators(pw, i, tables, currentCtx);
+ }
+ }
+
+ public AbstractValuesProcessor compile(RelNode plan) throws Exception {
+ String javaCode = generateJavaSource(plan);
+ LOG.debug("Compiling... source code {}", javaCode);
+ ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
+ PACKAGE_NAME + ".Processor",
+ javaCode, null);
+ return (AbstractValuesProcessor) cl.loadClass(
+ PACKAGE_NAME + ".Processor").newInstance();
+ }
+
+ private static void printEpilogue(
+ PrintWriter pw) throws Exception {
+ pw.print("}\n");
+ }
+
+ private static void printPrologue(PrintWriter pw) {
+ pw.append(PROLOGUE);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
new file mode 100644
index 0000000..afed8a9
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.*;
+import org.apache.calcite.rel.stream.Delta;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public abstract class PostOrderRelNodeVisitor<T> {
+ public final T traverse(RelNode n) throws Exception {
+ List<T> inputStreams = new ArrayList<>();
+ for (RelNode input : n.getInputs()) {
+ inputStreams.add(traverse(input));
+ }
+
+ if (n instanceof Aggregate) {
+ return visitAggregate((Aggregate) n, inputStreams);
+ } else if (n instanceof Calc) {
+ return visitCalc((Calc) n, inputStreams);
+ } else if (n instanceof Collect) {
+ return visitCollect((Collect) n, inputStreams);
+ } else if (n instanceof Correlate) {
+ return visitCorrelate((Correlate) n, inputStreams);
+ } else if (n instanceof Delta) {
+ return visitDelta((Delta) n, inputStreams);
+ } else if (n instanceof Exchange) {
+ return visitExchange((Exchange) n, inputStreams);
+ } else if (n instanceof Project) {
+ return visitProject((Project) n, inputStreams);
+ } else if (n instanceof Filter) {
+ return visitFilter((Filter) n, inputStreams);
+ } else if (n instanceof Sample) {
+ return visitSample((Sample) n, inputStreams);
+ } else if (n instanceof Sort) {
+ return visitSort((Sort) n, inputStreams);
+ } else if (n instanceof TableModify) {
+ return visitTableModify((TableModify) n, inputStreams);
+ } else if (n instanceof TableScan) {
+ return visitTableScan((TableScan) n, inputStreams);
+ } else if (n instanceof Uncollect) {
+ return visitUncollect((Uncollect) n, inputStreams);
+ } else if (n instanceof Window) {
+ return visitWindow((Window) n, inputStreams);
+ } else if (n instanceof Join) {
+ return visitJoin((Join) n, inputStreams);
+ } else {
+ return defaultValue(n, inputStreams);
+ }
+ }
+
+ public T visitAggregate(Aggregate aggregate, List<T> inputStreams) throws Exception {
+ return defaultValue(aggregate, inputStreams);
+ }
+
+ public T visitCalc(Calc calc, List<T> inputStreams) throws Exception {
+ return defaultValue(calc, inputStreams);
+ }
+
+ public T visitCollect(Collect collect, List<T> inputStreams) throws Exception {
+ return defaultValue(collect, inputStreams);
+ }
+
+ public T visitCorrelate(Correlate correlate, List<T> inputStreams) throws Exception {
+ return defaultValue(correlate, inputStreams);
+ }
+
+ public T visitDelta(Delta delta, List<T> inputStreams) throws Exception {
+ return defaultValue(delta, inputStreams);
+ }
+
+ public T visitExchange(Exchange exchange, List<T> inputStreams) throws Exception {
+ return defaultValue(exchange, inputStreams);
+ }
+
+ public T visitProject(Project project, List<T> inputStreams) throws Exception {
+ return defaultValue(project, inputStreams);
+ }
+
+ public T visitFilter(Filter filter, List<T> inputStreams) throws Exception {
+ return defaultValue(filter, inputStreams);
+ }
+
+ public T visitSample(Sample sample, List<T> inputStreams) throws Exception {
+ return defaultValue(sample, inputStreams);
+ }
+
+ public T visitSort(Sort sort, List<T> inputStreams) throws Exception {
+ return defaultValue(sort, inputStreams);
+ }
+
+ public T visitTableModify(TableModify modify, List<T> inputStreams) throws Exception {
+ return defaultValue(modify, inputStreams);
+ }
+
+ public T visitTableScan(TableScan scan, List<T> inputStreams) throws Exception {
+ return defaultValue(scan, inputStreams);
+ }
+
+ public T visitUncollect(Uncollect uncollect, List<T> inputStreams) throws Exception {
+ return defaultValue(uncollect, inputStreams);
+ }
+
+ public T visitWindow(Window window, List<T> inputStreams) throws Exception {
+ return defaultValue(window, inputStreams);
+ }
+
+ public T visitJoin(Join join, List<T> inputStreams) throws Exception {
+ return defaultValue(join, inputStreams);
+ }
+
+ public T defaultValue(RelNode n, List<T> inputStreams) {
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
new file mode 100644
index 0000000..97995c7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
@@ -0,0 +1,484 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Compile RelNodes into individual functions.
+ */
+class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
+ public static Joiner NEW_LINE_JOINER = Joiner.on('\n');
+
+ private final PrintWriter pw;
+ private final JavaTypeFactory typeFactory;
+ private final RexNodeToJavaCodeCompiler rexCompiler;
+
+ private static final String STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+ " private static final ChannelHandler %1$s = ",
+ " new AbstractChannelHandler() {",
+ " @Override",
+ " public void dataReceived(ChannelContext ctx, Values _data) {",
+ ""
+ );
+
+ private static final String AGGREGATE_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+ " private static final ChannelHandler %1$s = ",
+ " new AbstractChannelHandler() {",
+ " private final Values EMPTY_VALUES = new Values();",
+ " private final Map<List<Object>, Map<String, Object>> state = new LinkedHashMap<>();",
+ " private final int[] groupIndices = new int[] {%2$s};",
+ " private List<Object> getGroupValues(Values _data) {",
+ " List<Object> res = new ArrayList<>();",
+ " for (int i: groupIndices) {",
+ " res.add(_data.get(i));",
+ " }",
+ " return res;",
+ " }",
+ "",
+ " @Override",
+ " public void flush(ChannelContext ctx) {",
+ " emitAggregateResults(ctx);",
+ " super.flush(ctx);",
+ " state.clear();",
+ " }",
+ "",
+ " private void emitAggregateResults(ChannelContext ctx) {",
+ " for (Map.Entry<List<Object>, Map<String, Object>> entry: state.entrySet()) {",
+ " List<Object> groupValues = entry.getKey();",
+ " Map<String, Object> accumulators = entry.getValue();",
+ " %3$s",
+ " }",
+ " }",
+ "",
+ " @Override",
+ " public void dataReceived(ChannelContext ctx, Values _data) {",
+ ""
+ );
+
+ private static final String JOIN_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+ " private static final ChannelHandler %1$s = ",
+ " new AbstractChannelHandler() {",
+ " Object left = %2$s;",
+ " Object right = %3$s;",
+ " Object source = null;",
+ " List<Values> leftRows = new ArrayList<>();",
+ " List<Values> rightRows = new ArrayList<>();",
+ " boolean leftDone = false;",
+ " boolean rightDone = false;",
+ " int[] ordinals = new int[] {%4$s, %5$s};",
+ "",
+ " Multimap<Object, Values> getJoinTable(List<Values> rows, int joinIndex) {",
+ " Multimap<Object, Values> m = ArrayListMultimap.create();",
+ " for(Values v: rows) {",
+ " m.put(v.get(joinIndex), v);",
+ " }",
+ " return m;",
+ " }",
+ "",
+ " List<Values> join(Multimap<Object, Values> tab, List<Values> rows, int rowIdx, boolean rev) {",
+ " List<Values> res = new ArrayList<>();",
+ " for (Values row: rows) {",
+ " for (Values mapValue: tab.get(row.get(rowIdx))) {",
+ " if (mapValue != null) {",
+ " Values joinedRow = new Values();",
+ " if(rev) {",
+ " joinedRow.addAll(row);",
+ " joinedRow.addAll(mapValue);",
+ " } else {",
+ " joinedRow.addAll(mapValue);",
+ " joinedRow.addAll(row);",
+ " }",
+ " res.add(joinedRow);",
+ " }",
+ " }",
+ " }",
+ " return res;",
+ " }",
+ "",
+ " @Override",
+ " public void setSource(ChannelContext ctx, Object source) {",
+ " this.source = source;",
+ " }",
+ "",
+ " @Override",
+ " public void flush(ChannelContext ctx) {",
+ " if (source == left) {",
+ " leftDone = true;",
+ " } else if (source == right) {",
+ " rightDone = true;",
+ " }",
+ " if (leftDone && rightDone) {",
+ " if (leftRows.size() <= rightRows.size()) {",
+ " for(Values res: join(getJoinTable(leftRows, ordinals[0]), rightRows, ordinals[1], false)) {",
+ " ctx.emit(res);",
+ " }",
+ " } else {",
+ " for(Values res: join(getJoinTable(rightRows, ordinals[1]), leftRows, ordinals[0], true)) {",
+ " ctx.emit(res);",
+ " }",
+ " }",
+ " leftDone = rightDone = false;",
+ " leftRows.clear();",
+ " rightRows.clear();",
+ " super.flush(ctx);",
+ " }",
+ " }",
+ "",
+ " @Override",
+ " public void dataReceived(ChannelContext ctx, Values _data) {",
+ ""
+ );
+
+ private static final String STAGE_PASSTHROUGH = NEW_LINE_JOINER.join(
+ " private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;",
+ "");
+
+ private static final String STAGE_ENUMERABLE_TABLE_SCAN = NEW_LINE_JOINER.join(
+ " private static final ChannelHandler %1$s = new AbstractChannelHandler() {",
+ " @Override",
+ " public void flush(ChannelContext ctx) {",
+ " ctx.setSource(this);",
+ " super.flush(ctx);",
+ " }",
+ "",
+ " @Override",
+ " public void dataReceived(ChannelContext ctx, Values _data) {",
+ " ctx.setSource(this);",
+ " ctx.emit(_data);",
+ " }",
+ " };",
+ "");
+
+ private int nameCount;
+ private Map<AggregateCall, String> aggregateCallVarNames = new HashMap<>();
+
+ RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+ this.pw = pw;
+ this.typeFactory = typeFactory;
+ this.rexCompiler = new RexNodeToJavaCodeCompiler(new RexBuilder(typeFactory));
+ }
+
+ @Override
+ public Void visitDelta(Delta delta, List<Void> inputStreams) throws Exception {
+ pw.print(String.format(STAGE_PASSTHROUGH, getStageName(delta)));
+ return null;
+ }
+
+ @Override
+ public Void visitFilter(Filter filter, List<Void> inputStreams) throws Exception {
+ beginStage(filter);
+
+ List<RexNode> childExps = filter.getChildExps();
+ RelDataType inputRowType = filter.getInput(0).getRowType();
+
+ pw.print("Context context = new StormContext(Processor.dataContext);\n");
+ pw.print("context.values = _data.toArray();\n");
+ pw.print("Object[] outputValues = new Object[1];\n");
+
+ pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
+
+ String r = "((Boolean) outputValues[0])";
+ if (filter.getCondition().getType().isNullable()) {
+ pw.print(String.format(" if (%s != null && %s) { ctx.emit(_data); }\n", r, r));
+ } else {
+ pw.print(String.format(" if (%s) { ctx.emit(_data); }\n", r, r));
+ }
+ endStage();
+ return null;
+ }
+
+ @Override
+ public Void visitProject(Project project, List<Void> inputStreams) throws Exception {
+ beginStage(project);
+
+ List<RexNode> childExps = project.getChildExps();
+ RelDataType inputRowType = project.getInput(0).getRowType();
+ int outputCount = project.getRowType().getFieldCount();
+
+ pw.print("Context context = new StormContext(Processor.dataContext);\n");
+ pw.print("context.values = _data.toArray();\n");
+ pw.print(String.format("Object[] outputValues = new Object[%d];\n", outputCount));
+
+ pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
+
+ pw.print(" ctx.emit(new Values(outputValues));\n");
+ endStage();
+ return null;
+ }
+
+ @Override
+ public Void defaultValue(RelNode n, List<Void> inputStreams) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Void visitTableScan(TableScan scan, List<Void> inputStreams) throws Exception {
+ pw.print(String.format(STAGE_ENUMERABLE_TABLE_SCAN, getStageName(scan)));
+ return null;
+ }
+
+ @Override
+ public Void visitAggregate(Aggregate aggregate, List<Void> inputStreams) throws Exception {
+ beginAggregateStage(aggregate);
+ pw.println(" if (_data != null) {");
+ pw.println(" List<Object> curGroupValues = getGroupValues(_data);");
+ pw.println(" if (!state.containsKey(curGroupValues)) {");
+ pw.println(" state.put(curGroupValues, new HashMap<String, Object>());");
+ pw.println(" }");
+ pw.println(" Map<String, Object> accumulators = state.get(curGroupValues);");
+ for (AggregateCall call : aggregate.getAggCallList()) {
+ aggregate(call);
+ }
+ pw.println(" }");
+ endStage();
+ return null;
+ }
+
+ @Override
+ public Void visitJoin(Join join, List<Void> inputStreams) {
+ beginJoinStage(join);
+ pw.println(" if (source == left) {");
+ pw.println(" leftRows.add(_data);");
+ pw.println(" } else if (source == right) {");
+ pw.println(" rightRows.add(_data);");
+ pw.println(" }");
+ endStage();
+ return null;
+ }
+
+ private String groupValueEmitStr(String var, int n) {
+ int count = 0;
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < n; i++) {
+ if (++count > 1) {
+ sb.append(", ");
+ }
+ sb.append(var).append(".").append("get(").append(i).append(")");
+ }
+ return sb.toString();
+ }
+
+ private String emitAggregateStmts(Aggregate aggregate) {
+ List<String> res = new ArrayList<>();
+ StringWriter sw = new StringWriter();
+ for (AggregateCall call : aggregate.getAggCallList()) {
+ res.add(aggregateResult(call, new PrintWriter(sw)));
+ }
+ return NEW_LINE_JOINER.join(sw.toString(),
+ String.format(" ctx.emit(new Values(%s, %s));",
+ groupValueEmitStr("groupValues", aggregate.getGroupSet().cardinality()),
+ Joiner.on(", ").join(res)));
+ }
+
+ private String aggregateResult(AggregateCall call, PrintWriter pw) {
+ SqlAggFunction aggFunction = call.getAggregation();
+ String aggregationName = call.getAggregation().getName();
+ Type ty = typeFactory.getJavaClass(call.getType());
+ String result;
+ if (aggFunction instanceof SqlUserDefinedAggFunction) {
+ AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
+ result = doAggregateResult((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, pw);
+ } else {
+ List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
+ if (typeClasses == null) {
+ throw new UnsupportedOperationException(aggregationName + " Not implemented");
+ }
+ result = doAggregateResult(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
+ reserveAggVarName(call), ty, pw);
+ }
+ return result;
+ }
+
+ private String doAggregateResult(AggregateFunctionImpl aggFn, String varName, Type ty, PrintWriter pw) {
+ String resultName = varName + "_result";
+ Class<?> accumulatorType = aggFn.accumulatorType;
+ Class<?> resultType = aggFn.resultType;
+ List<String> args = new ArrayList<>();
+ if (!aggFn.isStatic) {
+ String aggObjName = String.format("%s_obj", varName);
+ String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
+ pw.println(" @SuppressWarnings(\"unchecked\")");
+ pw.println(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
+ aggObjName));
+ args.add(aggObjName);
+ }
+ args.add(String.format("(%s)accumulators.get(\"%s\")", accumulatorType.getCanonicalName(), varName));
+ pw.println(String.format(" final %s %s = %s;", resultType.getCanonicalName(),
+ resultName, printMethodCall(aggFn.resultMethod, args)));
+
+ return resultName;
+ }
+
+ private void aggregate(AggregateCall call) {
+ SqlAggFunction aggFunction = call.getAggregation();
+ String aggregationName = call.getAggregation().getName();
+ Type ty = typeFactory.getJavaClass(call.getType());
+ if (call.getArgList().size() != 1) {
+ if (aggregationName.equals("COUNT")) {
+ if (call.getArgList().size() != 0) {
+ throw new UnsupportedOperationException("Count with nullable fields");
+ }
+ }
+ }
+ if (aggFunction instanceof SqlUserDefinedAggFunction) {
+ AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
+ doAggregate((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, call.getArgList());
+ } else {
+ List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
+ if (typeClasses == null) {
+ throw new UnsupportedOperationException(aggregationName + " Not implemented");
+ }
+ doAggregate(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
+ reserveAggVarName(call), ty, call.getArgList());
+ }
+ }
+
+ private Class<?> findMatchingClass(String aggregationName, List<BuiltinAggregateFunctions.TypeClass> typeClasses, Type ty) {
+ for (BuiltinAggregateFunctions.TypeClass typeClass : typeClasses) {
+ if (typeClass.ty.equals(BuiltinAggregateFunctions.TypeClass.GenericType.class) || typeClass.ty.equals(ty)) {
+ return typeClass.clazz;
+ }
+ }
+ throw new UnsupportedOperationException(aggregationName + " Not implemeted for type '" + ty + "'");
+ }
+
+ private void doAggregate(AggregateFunctionImpl aggFn, String varName, Type ty, List<Integer> argList) {
+ List<String> args = new ArrayList<>();
+ Class<?> accumulatorType = aggFn.accumulatorType;
+ if (!aggFn.isStatic) {
+ String aggObjName = String.format("%s_obj", varName);
+ String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
+ pw.println(String.format(" if (!accumulators.containsKey(\"%s\")) { ", aggObjName));
+ pw.println(String.format(" accumulators.put(\"%s\", new %s());", aggObjName, aggObjClassName));
+ pw.println(" }");
+ pw.println(" @SuppressWarnings(\"unchecked\")");
+ pw.println(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
+ aggObjName));
+ args.add(aggObjName);
+ }
+ args.add(String.format("%1$s == null ? %2$s : (%3$s) %1$s",
+ "accumulators.get(\"" + varName + "\")",
+ printMethodCall(aggFn.initMethod, args),
+ accumulatorType.getCanonicalName()));
+ if (argList.isEmpty()) {
+ args.add("EMPTY_VALUES");
+ } else {
+ for (int i = 0; i < aggFn.valueTypes.size(); i++) {
+ args.add(String.format("(%s) %s", aggFn.valueTypes.get(i).getCanonicalName(), "_data.get(" + argList.get(i) + ")"));
+ }
+ }
+ pw.print(String.format(" accumulators.put(\"%s\", %s);\n",
+ varName,
+ printMethodCall(aggFn.addMethod, args)));
+ }
+
+ private String reserveAggVarName(AggregateCall call) {
+ String varName;
+ if ((varName = aggregateCallVarNames.get(call)) == null) {
+ varName = call.getAggregation().getName() + ++nameCount;
+ aggregateCallVarNames.put(call, varName);
+ }
+ return varName;
+ }
+
+ private void beginStage(RelNode n) {
+ pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
+ }
+
+ private void beginAggregateStage(Aggregate n) {
+ pw.print(String.format(AGGREGATE_STAGE_PROLOGUE, getStageName(n), getGroupByIndices(n), emitAggregateStmts(n)));
+ }
+
+ private void beginJoinStage(Join join) {
+ int[] ordinals = new int[2];
+ if (!RelOptUtil.analyzeSimpleEquiJoin((LogicalJoin) join, ordinals)) {
+ throw new UnsupportedOperationException("Only simple equi joins are supported");
+ }
+
+ pw.print(String.format(JOIN_STAGE_PROLOGUE, getStageName(join),
+ getStageName(join.getLeft()),
+ getStageName(join.getRight()),
+ ordinals[0],
+ ordinals[1]));
+ }
+
+ private void endStage() {
+ pw.print(" }\n };\n");
+ }
+
+ static String getStageName(RelNode n) {
+ return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
+ }
+
+ private String getGroupByIndices(Aggregate n) {
+ StringBuilder res = new StringBuilder();
+ int count = 0;
+ for (int i : n.getGroupSet()) {
+ if (++count > 1) {
+ res.append(", ");
+ }
+ res.append(i);
+ }
+ return res.toString();
+ }
+
+ public static String printMethodCall(Method method, List<String> args) {
+ return printMethodCall(method.getDeclaringClass(), method.getName(),
+ Modifier.isStatic(method.getModifiers()), args);
+ }
+
+ private static String printMethodCall(Class<?> clazz, String method, boolean isStatic, List<String> args) {
+ if (isStatic) {
+ return String.format("%s.%s(%s)", clazz.getCanonicalName(), method, Joiner.on(',').join(args));
+ } else {
+ return String.format("%s.%s(%s)", args.get(0), method,
+ Joiner.on(',').join(args.subList(1, args.size())));
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
new file mode 100644
index 0000000..0b7c053
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
@@ -0,0 +1,225 @@
+/*
+ * Copyright (C) 2010 Google, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.javac;
+
+
+import javax.tools.DiagnosticListener;
+import javax.tools.FileObject;
+import javax.tools.ForwardingJavaFileManager;
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileManager;
+import javax.tools.JavaFileObject;
+import javax.tools.SimpleJavaFileObject;
+import javax.tools.ToolProvider;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Collections.singleton;
+
+/**
+ * This is a Java ClassLoader that will attempt to load a class from a string of source code.
+ *
+ * <h3>Example</h3>
+ *
+ * <pre>
+ * String className = "com.foo.MyClass";
+ * String classSource =
+ * "package com.foo;\n" +
+ * "public class MyClass implements Runnable {\n" +
+ * " @Override public void run() {\n" +
+ * " log(\"Hello world\");\n" +
+ * " }\n" +
+ * "}";
+ *
+ * // Load class from source.
+ * ClassLoader classLoader = new CompilingClassLoader(
+ * parentClassLoader, className, classSource);
+ * Class myClass = classLoader.loadClass(className);
+ *
+ * // Use it.
+ * Runnable instance = (Runnable)myClass.newInstance();
+ * instance.run();
+ * </pre>
+ *
+ * Only one chunk of source can be compiled per instance of CompilingClassLoader. If you need to
+ * compile more, create multiple CompilingClassLoader instances.
+ *
+ * Uses Java 1.6's in built compiler API.
+ *
+ * If the class cannot be compiled, loadClass() will throw a ClassNotFoundException and log the
+ * compile errors to System.err. If you don't want the messages logged, or want to explicitly handle
+ * the messages you can provide your own {@link javax.tools.DiagnosticListener} through
+ * {#setDiagnosticListener()}.
+ *
+ * @see java.lang.ClassLoader
+ * @see javax.tools.JavaCompiler
+ */
+public class CompilingClassLoader extends ClassLoader {
+
+ /**
+ * Thrown when code cannot be compiled.
+ */
+ public static class CompilerException extends Exception {
+ private static final long serialVersionUID = -2936958840023603270L;
+
+ public CompilerException(String message) {
+ super(message);
+ }
+ }
+
+ private final Map<String, ByteArrayOutputStream> byteCodeForClasses = new HashMap<>();
+
+ private static final URI EMPTY_URI;
+
+ static {
+ try {
+ // Needed to keep SimpleFileObject constructor happy.
+ EMPTY_URI = new URI("");
+ } catch (URISyntaxException e) {
+ throw new Error(e);
+ }
+ }
+
+ /**
+ * @param parent Parent classloader to resolve dependencies from.
+ * @param className Name of class to compile. eg. "com.foo.MyClass".
+ * @param sourceCode Java source for class. e.g. "package com.foo; class MyClass { ... }".
+ * @param diagnosticListener Notified of compiler errors (may be null).
+ */
+ public CompilingClassLoader(
+ ClassLoader parent,
+ String className,
+ String sourceCode,
+ DiagnosticListener<JavaFileObject> diagnosticListener)
+ throws CompilerException {
+ super(parent);
+ if (!compileSourceCodeToByteCode(className, sourceCode, diagnosticListener)) {
+ throw new CompilerException("Could not compile " + className);
+ }
+ }
+
+ public Map<String, ByteArrayOutputStream> getClasses() {
+ return byteCodeForClasses;
+ }
+
+ /**
+ * Override ClassLoader's class resolving method. Don't call this directly, instead use
+ * {@link ClassLoader#loadClass(String)}.
+ */
+ @Override
+ public Class<?> findClass(String name) throws ClassNotFoundException {
+ ByteArrayOutputStream byteCode = byteCodeForClasses.get(name);
+ if (byteCode == null) {
+ throw new ClassNotFoundException(name);
+ }
+ return defineClass(name, byteCode.toByteArray(), 0, byteCode.size());
+ }
+
+ /**
+ * @return Whether compilation was successful.
+ */
+ private boolean compileSourceCodeToByteCode(
+ String className, String sourceCode, DiagnosticListener<JavaFileObject> diagnosticListener) {
+ JavaCompiler javaCompiler = ToolProvider.getSystemJavaCompiler();
+
+ // Set up the in-memory filesystem.
+ InMemoryFileManager fileManager =
+ new InMemoryFileManager(javaCompiler.getStandardFileManager(null, null, null));
+ JavaFileObject javaFile = new InMemoryJavaFile(className, sourceCode);
+
+ // Javac option: remove these when the javac zip impl is fixed
+ // (http://b/issue?id=1822932)
+ System.setProperty("useJavaUtilZip", "true"); // setting value to any non-null string
+ List<String> options = new LinkedList<>();
+ // this is ignored by javac currently but useJavaUtilZip should be
+ // a valid javac XD option, which is another bug
+ options.add("-XDuseJavaUtilZip");
+
+ // Now compile!
+ JavaCompiler.CompilationTask compilationTask =
+ javaCompiler.getTask(
+ null, // Null: log any unhandled errors to stderr.
+ fileManager,
+ diagnosticListener,
+ options,
+ null,
+ singleton(javaFile));
+ return compilationTask.call();
+ }
+
+ /**
+ * Provides an in-memory representation of JavaFileManager abstraction, so we do not need to write
+ * any files to disk.
+ *
+ * When files are written to, rather than putting the bytes on disk, they are appended to buffers
+ * in byteCodeForClasses.
+ *
+ * @see javax.tools.JavaFileManager
+ */
+ private class InMemoryFileManager extends ForwardingJavaFileManager<JavaFileManager> {
+ public InMemoryFileManager(JavaFileManager fileManager) {
+ super(fileManager);
+ }
+
+ @Override
+ public JavaFileObject getJavaFileForOutput(
+ Location location, final String className, JavaFileObject.Kind kind, FileObject sibling)
+ throws IOException {
+ return new SimpleJavaFileObject(EMPTY_URI, kind) {
+ @Override
+ public OutputStream openOutputStream() throws IOException {
+ ByteArrayOutputStream outputStream = byteCodeForClasses.get(className);
+ if (outputStream != null) {
+ throw new IllegalStateException("Cannot write more than once");
+ }
+ // Reasonable size for a simple .class.
+ outputStream = new ByteArrayOutputStream(256);
+ byteCodeForClasses.put(className, outputStream);
+ return outputStream;
+ }
+ };
+ }
+ }
+
+ private static class InMemoryJavaFile extends SimpleJavaFileObject {
+ private final String sourceCode;
+
+ public InMemoryJavaFile(String className, String sourceCode) {
+ super(makeUri(className), Kind.SOURCE);
+ this.sourceCode = sourceCode;
+ }
+
+ private static URI makeUri(String className) {
+ try {
+ return new URI(className.replaceAll("\\.", "/") + Kind.SOURCE.extension);
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e); // Not sure what could cause this.
+ }
+ }
+
+ @Override
+ public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
+ return sourceCode;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
new file mode 100644
index 0000000..c67d8e7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+
+public class ColumnConstraint extends SqlLiteral {
+ private ColumnConstraint(
+ Object value, SqlTypeName typeName, SqlParserPos pos) {
+ super(value, typeName, pos);
+ }
+
+ public static class PrimaryKey extends ColumnConstraint {
+ private final SqlMonotonicity monotonicity;
+ public PrimaryKey(SqlMonotonicity monotonicity, SqlParserPos pos) {
+ super(SqlDDLKeywords.PRIMARY, SqlTypeName.SYMBOL, pos);
+ this.monotonicity = monotonicity;
+ }
+ public SqlMonotonicity monotonicity() {
+ return monotonicity;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
new file mode 100644
index 0000000..3520b86
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import java.util.Arrays;
+
+public class ColumnDefinition extends SqlNodeList {
+ public ColumnDefinition(
+ SqlIdentifier name, SqlDataTypeSpec type, ColumnConstraint constraint, SqlParserPos pos) {
+ super(Arrays.asList(name, type, constraint), pos);
+ }
+
+ public String name() {
+ return get(0).toString();
+ }
+
+ public SqlDataTypeSpec type() {
+ return (SqlDataTypeSpec) get(1);
+ }
+
+ public ColumnConstraint constraint() {
+ return (ColumnConstraint) get(2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
new file mode 100644
index 0000000..a53802c
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.NlsString;
+
+import java.util.List;
+
+public class SqlCreateFunction extends SqlCall {
+ public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
+ "CREATE_FUNCTION", SqlKind.OTHER) {
+ @Override
+ public SqlCall createCall(
+ SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
+ assert functionQualifier == null;
+ return new SqlCreateFunction(pos, (SqlIdentifier) o[0], o[1], o[2]);
+ }
+
+ @Override
+ public void unparse(
+ SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+ SqlCreateFunction t = (SqlCreateFunction) call;
+ UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
+ u.keyword("CREATE", "FUNCTION").node(t.functionName).keyword("AS").node(t.className);
+ if (t.jarName != null) {
+ u.keyword("USING", "JAR").node(t.jarName);
+ }
+ }
+ };
+
+ private final SqlIdentifier functionName;
+ private final SqlNode className;
+ private final SqlNode jarName;
+
+ public SqlCreateFunction(SqlParserPos pos, SqlIdentifier functionName, SqlNode className, SqlNode jarName) {
+ super(pos);
+ this.functionName = functionName;
+ this.className = className;
+ this.jarName = jarName;
+ }
+
+ @Override
+ public SqlOperator getOperator() {
+ return OPERATOR;
+ }
+
+ @Override
+ public List<SqlNode> getOperandList() {
+ return ImmutableNullableList.of(functionName, className);
+ }
+
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ getOperator().unparse(writer, this, leftPrec, rightPrec);
+ }
+
+ public String functionName() {
+ return functionName.toString();
+ }
+
+ public String className() {
+ return ((NlsString)SqlLiteral.value(className)).getValue();
+ }
+
+ public String jarName() {
+ return jarName == null ? null : ((NlsString)SqlLiteral.value(jarName)).getValue();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
new file mode 100644
index 0000000..670eedb
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Properties;
+
+public class SqlCreateTable extends SqlCall {
+ private static final int DEFAULT_PARALLELISM = 1;
+
+ public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
+ "CREATE_TABLE", SqlKind.OTHER) {
+ @Override
+ public SqlCall createCall(
+ SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
+ assert functionQualifier == null;
+ return new SqlCreateTable(pos, (SqlIdentifier) o[0], (SqlNodeList) o[1],
+ o[2], o[3], o[4], o[5], o[6], o[7]);
+ }
+
+ @Override
+ public void unparse(
+ SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+ SqlCreateTable t = (SqlCreateTable) call;
+ UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
+ u.keyword("CREATE", "EXTERNAL", "TABLE").node(t.tblName).nodeList(
+ t.fieldList);
+ if (t.inputFormatClass != null && t.outputFormatClass != null) {
+ u.keyword("STORED", "AS", "INPUTFORMAT").node(
+ t.inputFormatClass).keyword("OUTPUTFORMAT").node(
+ t.outputFormatClass);
+ }
+ u.keyword("LOCATION").node(t.location);
+ if (t.parallelism != null) {
+ u.keyword("PARALLELISM").node(t.parallelism);
+ }
+ if (t.properties != null) {
+ u.keyword("TBLPROPERTIES").node(t.properties);
+ }
+ if (t.query != null) {
+ u.keyword("AS").node(t.query);
+ }
+ }
+ };
+
+ private final SqlIdentifier tblName;
+ private final SqlNodeList fieldList;
+ private final SqlNode inputFormatClass;
+ private final SqlNode outputFormatClass;
+ private final SqlNode location;
+ private final SqlNode parallelism;
+ private final SqlNode properties;
+ private final SqlNode query;
+
+ public SqlCreateTable(
+ SqlParserPos pos, SqlIdentifier tblName, SqlNodeList fieldList,
+ SqlNode inputFormatClass, SqlNode outputFormatClass, SqlNode location,
+ SqlNode parallelism, SqlNode properties, SqlNode query) {
+ super(pos);
+ this.tblName = tblName;
+ this.fieldList = fieldList;
+ this.inputFormatClass = inputFormatClass;
+ this.outputFormatClass = outputFormatClass;
+ this.location = location;
+ this.parallelism = parallelism;
+ this.properties = properties;
+ this.query = query;
+ }
+
+ @Override
+ public SqlOperator getOperator() {
+ return OPERATOR;
+ }
+
+ @Override
+ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+ getOperator().unparse(writer, this, leftPrec, rightPrec);
+ }
+
+ @Override
+ public List<SqlNode> getOperandList() {
+ return ImmutableNullableList.of(tblName, fieldList, inputFormatClass,
+ outputFormatClass, location, properties,
+ query);
+ }
+
+ public String tableName() {
+ return tblName.toString();
+ }
+
+ public URI location() {
+ return URI.create(getString(location));
+ }
+
+ public Integer parallelism() {
+ String parallelismStr = getString(parallelism);
+ if (parallelismStr != null) {
+ return Integer.parseInt(parallelismStr);
+ } else {
+ return DEFAULT_PARALLELISM;
+ }
+ }
+
+ public String inputFormatClass() {
+ return getString(inputFormatClass);
+ }
+
+ public String outputFormatClass() {
+ return getString(outputFormatClass);
+ }
+
+ public Properties properties() {
+ Properties props = new Properties();
+ if (properties != null) {
+ try {
+ ObjectMapper mapper = new ObjectMapper();
+ HashMap<String, Object> map = mapper.readValue(getString(properties), HashMap.class);
+ props.putAll(map);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return props;
+ }
+
+ private String getString(SqlNode n) {
+ return n == null ? null : SqlLiteral.stringValue(n);
+ }
+
+ @SuppressWarnings("unchecked")
+ public List<ColumnDefinition> fieldList() {
+ return (List<ColumnDefinition>)((List<? extends SqlNode>)fieldList.getList());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
new file mode 100644
index 0000000..3112e53
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlLiteral;
+
+/**
+ * Define the keywords that can occur in a CREATE TABLE statement
+ */
+public enum SqlDDLKeywords implements SqlLiteral.SqlSymbol {
+ PRIMARY
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
new file mode 100644
index 0000000..8444e1e
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.calcite.config.Lex;
+import org.apache.storm.sql.parser.impl.StormParserImpl;
+
+import java.io.StringReader;
+
+public class StormParser {
+ public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
+ private final StormParserImpl impl;
+
+ public StormParser(String s) {
+ this.impl = new StormParserImpl(new StringReader(s));
+ this.impl.setTabSize(1);
+ this.impl.setQuotedCasing(Lex.ORACLE.quotedCasing);
+ this.impl.setUnquotedCasing(Lex.ORACLE.unquotedCasing);
+ this.impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH);
+ /*
+ * By default parser uses [ ] for quoting identifiers. Switching to DQID (double quoted identifiers)
+ * is needed for array and map access (m['x'] = 1 or arr[2] = 10 etc) to work.
+ */
+ this.impl.switchTo("DQID");
+ }
+
+ @VisibleForTesting
+ public StormParserImpl impl() {
+ return impl;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
new file mode 100644
index 0000000..8e0a1d9
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWriter;
+
+class UnparseUtil {
+ private final SqlWriter writer;
+ private final int leftPrec;
+ private final int rightPrec;
+
+ UnparseUtil(SqlWriter writer, int leftPrec, int rightPrec) {
+ this.writer = writer;
+ this.leftPrec = leftPrec;
+ this.rightPrec = rightPrec;
+ }
+
+ UnparseUtil keyword(String... keywords) {
+ for (String k : keywords) {
+ writer.keyword(k);
+ }
+ return this;
+ }
+
+ UnparseUtil node(SqlNode n) {
+ n.unparse(writer, leftPrec, rightPrec);
+ return this;
+ }
+
+ UnparseUtil nodeList(SqlNodeList l) {
+ writer.keyword("(");
+ if (l.size() > 0) {
+ l.get(0).unparse(writer, leftPrec, rightPrec);
+ for (int i = 1; i < l.size(); ++i) {
+ writer.keyword(",");
+ l.get(i).unparse(writer, leftPrec, rightPrec);
+ }
+ }
+ writer.keyword(")");
+ return this;
+ }
+}
[10/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/README.md
----------------------------------------------------------------------
diff --git a/flux/README.md b/flux/README.md
new file mode 100644
index 0000000..206ae22
--- /dev/null
+++ b/flux/README.md
@@ -0,0 +1,872 @@
+# flux
+A framework for creating and deploying Apache Storm streaming computations with less friction.
+
+## Definition
+**flux** |fl\u0259ks| _noun_
+
+1. The action or process of flowing or flowing out
+2. Continuous change
+3. In physics, the rate of flow of a fluid, radiant energy, or particles across a given area
+4. A substance mixed with a solid to lower its melting point
+
+## Rationale
+Bad things happen when configuration is hard-coded. No one should have to recompile or repackage an application in
+order to change configuration.
+
+## About
+Flux is a framework and set of utilities that make defining and deploying Apache Storm topologies less painful and
+deveoper-intensive.
+
+Have you ever found yourself repeating this pattern?:
+
+```java
+
+public static void main(String[] args) throws Exception {
+ // logic to determine if we're running locally or not...
+ // create necessary config options...
+ boolean runLocal = shouldRunLocal();
+ if(runLocal){
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology(name, conf, topology);
+ } else {
+ StormSubmitter.submitTopology(name, conf, topology);
+ }
+}
+```
+
+Wouldn't something like this be easier:
+
+```bash
+storm jar mytopology.jar org.apache.storm.flux.Flux --local config.yaml
+```
+
+or:
+
+```bash
+storm jar mytopology.jar org.apache.storm.flux.Flux --remote config.yaml
+```
+
+Another pain point often mentioned is the fact that the wiring for a Topology graph is often tied up in Java code,
+and that any changes require recompilation and repackaging of the topology jar file. Flux aims to alleviate that
+pain by allowing you to package all your Storm components in a single jar, and use an external text file to define
+the layout and configuration of your topologies.
+
+## Features
+
+ * Easily configure and deploy Storm topologies (Both Storm core and Microbatch API) without embedding configuration
+ in your topology code
+ * Support for existing topology code (see below)
+ * Define Storm Core API (Spouts/Bolts) using a flexible YAML DSL
+ * YAML DSL support for most Storm components (storm-kafka, storm-hdfs, storm-hbase, etc.)
+ * Convenient support for multi-lang components
+ * External property substitution/filtering for easily switching between configurations/environments (similar to Maven-style
+ `${variable.name}` substitution)
+
+## Usage
+
+To use Flux, add it as a dependency and package all your Storm components in a fat jar, then create a YAML document
+to define your topology (see below for YAML configuration options).
+
+### Building from Source
+The easiest way to use Flux, is to add it as a Maven dependency in you project as described below.
+
+If you would like to build Flux from source and run the unit/integration tests, you will need the following installed
+on your system:
+
+* Python 2.6.x or later
+* Node.js 0.10.x or later
+
+#### Building with unit tests enabled:
+
+```
+mvn clean install
+```
+
+#### Building with unit tests disabled:
+If you would like to build Flux without installing Python or Node.js you can simply skip the unit tests:
+
+```
+mvn clean install -DskipTests=true
+```
+
+Note that if you plan on using Flux to deploy topologies to a remote cluster, you will still need to have Python
+installed since it is required by Apache Storm.
+
+
+#### Building with integration tests enabled:
+
+```
+mvn clean install -DskipIntegration=false
+```
+
+
+### Packaging with Maven
+To enable Flux for your Storm components, you need to add it as a dependency such that it's included in the Storm
+topology jar. This can be accomplished with the Maven shade plugin (preferred) or the Maven assembly plugin (not
+recommended).
+
+#### Flux Maven Dependency
+The current version of Flux is available in Maven Central at the following coordinates:
+```xml
+<dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-core</artifactId>
+ <version>${storm.version}</version>
+</dependency>
+```
+
+Using shell spouts and bolts requires additional Flux Wrappers library:
+```xml
+<dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-wrappers</artifactId>
+ <version>${storm.version}</version>
+</dependency>
+```
+
+#### Creating a Flux-Enabled Topology JAR
+The example below illustrates Flux usage with the Maven shade plugin:
+
+ ```xml
+<!-- include Flux and user dependencies in the shaded jar -->
+<dependencies>
+ <!-- Flux include -->
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-core</artifactId>
+ <version>${storm.version}</version>
+ </dependency>
+ <!-- Flux Wrappers include -->
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-wrappers</artifactId>
+ <version>${storm.version}</version>
+ </dependency>
+
+ <!-- add user dependencies here... -->
+
+</dependencies>
+<!-- create a fat jar that includes all dependencies -->
+<build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>1.4</version>
+ <configuration>
+ <createDependencyReducedPom>true</createDependencyReducedPom>
+ </configuration>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <transformers>
+ <transformer
+ implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+ <transformer
+ implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+ <mainClass>org.apache.storm.flux.Flux</mainClass>
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+</build>
+ ```
+
+### Deploying and Running a Flux Topology
+Once your topology components are packaged with the Flux dependency, you can run different topologies either locally
+or remotely using the `storm jar` command. For example, if your fat jar is named `myTopology-0.1.0-SNAPSHOT.jar` you
+could run it locally with the command:
+
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml
+
+```
+
+### Command line options
+```
+usage: storm jar <my_topology_uber_jar.jar> org.apache.storm.flux.Flux
+ [options] <topology-config.yaml>
+ -d,--dry-run Do not run or deploy the topology. Just
+ build, validate, and print information about
+ the topology.
+ -e,--env-filter Perform environment variable substitution.
+ Replace keys identified with `${ENV-[NAME]}`
+ will be replaced with the corresponding
+ `NAME` environment value
+ -f,--filter <file> Perform property substitution. Use the
+ specified file as a source of properties,
+ and replace keys identified with {$[property
+ name]} with the value defined in the
+ properties file.
+ -i,--inactive Deploy the topology, but do not activate it.
+ -l,--local Run the topology in local mode.
+ -n,--no-splash Suppress the printing of the splash screen.
+ -q,--no-detail Suppress the printing of topology details.
+ -r,--remote Deploy the topology to a remote cluster.
+ -R,--resource Treat the supplied path as a classpath
+ resource instead of a file.
+ -s,--sleep <ms> When running locally, the amount of time to
+ sleep (in ms.) before killing the topology
+ and shutting down the local cluster.
+ -z,--zookeeper <host:port> When running in local mode, use the
+ ZooKeeper at the specified <host>:<port>
+ instead of the in-process ZooKeeper.
+ (requires Storm 0.9.3 or later)
+```
+
+**NOTE:** Flux tries to avoid command line switch collision with the `storm` command, and allows any other command line
+switches to pass through to the `storm` command.
+
+For example, you can use the `storm` command switch `-c` to override a topology configuration property. The following
+example command will run Flux and override the `nimus.host` configuration:
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --remote my_config.yaml -c nimbus.host=localhost
+```
+
+### Sample output
+```
+\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557 \u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557
+\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
+\u2588\u2588\u2588\u2588\u2588\u2557 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
+\u2588\u2588\u2554\u2550\u2550\u255d \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
+\u2588\u2588\u2551 \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
+\u255a\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d \u255a\u2550\u255d
++- Apache Storm -+
++- data FLow User eXperience -+
+Version: 0.3.0
+Parsing file: /Users/hsimpson/Projects/donut_domination/storm/shell_test.yaml
+---------- TOPOLOGY DETAILS ----------
+Name: shell-topology
+--------------- SPOUTS ---------------
+sentence-spout[1](org.apache.storm.flux.wrappers.spouts.FluxShellSpout)
+---------------- BOLTS ---------------
+splitsentence[1](org.apache.storm.flux.wrappers.bolts.FluxShellBolt)
+log[1](org.apache.storm.flux.wrappers.bolts.LogInfoBolt)
+count[1](org.apache.storm.testing.TestWordCounter)
+--------------- STREAMS ---------------
+sentence-spout --SHUFFLE--> splitsentence
+splitsentence --FIELDS--> count
+count --SHUFFLE--> log
+--------------------------------------
+Submitting topology: 'shell-topology' to remote cluster...
+```
+
+## YAML Configuration
+Flux topologies are defined in a YAML file that describes a topology. A Flux topology
+definition consists of the following:
+
+ 1. A topology name
+ 2. A list of topology "components" (named Java objects that will be made available in the environment)
+ 3. **EITHER** (A DSL topology definition):
+ * A list of spouts, each identified by a unique ID
+ * A list of bolts, each identified by a unique ID
+ * A list of "stream" objects representing a flow of tuples between spouts and bolts
+ 4. **OR** (A JVM class that can produce a `org.apache.storm.generated.StormTopology` instance:
+ * A `topologySource` definition.
+
+
+
+For example, here is a simple definition of a wordcount topology using the YAML DSL:
+
+```yaml
+name: "yaml-topology"
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ - id: "bolt-2"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+#stream definitions
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "bolt-1 --> bolt2"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: SHUFFLE
+
+
+```
+## Property Substitution/Filtering
+It's common for developers to want to easily switch between configurations, for example switching deployment between
+a development environment and a production environment. This can be accomplished by using separate YAML configuration
+files, but that approach would lead to unnecessary duplication, especially in situations where the Storm topology
+does not change, but configuration settings such as host names, ports, and parallelism paramters do.
+
+For this case, Flux offers properties filtering to allow you two externalize values to a `.properties` file and have
+them substituted before the `.yaml` file is parsed.
+
+To enable property filtering, use the `--filter` command line option and specify a `.properties` file. For example,
+if you invoked flux like so:
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml --filter dev.properties
+```
+With the following `dev.properties` file:
+
+```properties
+kafka.zookeeper.hosts: localhost:2181
+```
+
+You would then be able to reference those properties by key in your `.yaml` file using `${}` syntax:
+
+```yaml
+ - id: "zkHosts"
+ className: "org.apache.storm.kafka.ZkHosts"
+ constructorArgs:
+ - "${kafka.zookeeper.hosts}"
+```
+
+In this case, Flux would replace `${kafka.zookeeper.hosts}` with `localhost:2181` before parsing the YAML contents.
+
+### Environment Variable Substitution/Filtering
+Flux also allows environment variable substitution. For example, if an environment variable named `ZK_HOSTS` if defined,
+you can reference it in a Flux YAML file with the following syntax:
+
+```
+${ENV-ZK_HOSTS}
+```
+
+## Components
+Components are essentially named object instances that are made available as configuration options for spouts and
+bolts. If you are familiar with the Spring framework, components are roughly analagous to Spring beans.
+
+Every component is identified, at a minimum, by a unique identifier (String) and a class name (String). For example,
+the following will make an instance of the `org.apache.storm.kafka.StringScheme` class available as a reference under the key
+`"stringScheme"` . This assumes the `org.apache.storm.kafka.StringScheme` has a default constructor.
+
+```yaml
+components:
+ - id: "stringScheme"
+ className: "org.apache.storm.kafka.StringScheme"
+```
+
+### Contructor Arguments, References, Properties and Configuration Methods
+
+####Constructor Arguments
+Arguments to a class constructor can be configured by adding a `contructorArgs` element to a components.
+`constructorArgs` is a list of objects that will be passed to the class' constructor. The following example creates an
+object by calling the constructor that takes a single string as an argument:
+
+```yaml
+ - id: "zkHosts"
+ className: "org.apache.storm.kafka.ZkHosts"
+ constructorArgs:
+ - "localhost:2181"
+ - true
+```
+
+####References
+Each component instance is identified by a unique id that allows it to be used/reused by other components. To
+reference an existing component, you specify the id of the component with the `ref` tag.
+
+In the following example, a component with the id `"stringScheme"` is created, and later referenced, as a an argument
+to another component's constructor:
+
+```yaml
+components:
+ - id: "stringScheme"
+ className: "org.apache.storm.kafka.StringScheme"
+
+ - id: "stringMultiScheme"
+ className: "org.apache.storm.spout.SchemeAsMultiScheme"
+ constructorArgs:
+ - ref: "stringScheme" # component with id "stringScheme" must be declared above.
+```
+
+You can also reference existing components in list via specifying the id of the components with the `reflist` tag.
+The type of the reflist will be `List<Object>`, but Flux can automatically convert List to Array (also varargs),
+so you can use reflist on argument which type is `List<Type>`, or `Type[]`, or `Type...`.
+
+Please note that all components in the list must be same type.
+
+```yaml
+components:
+ - id: "boundCQLStatementMapperBuilder"
+ className: "org.apache.storm.cassandra.query.builder.BoundCQLStatementMapperBuilder"
+ constructorArgs:
+ - "INSERT INTO sink_cassandra (eventKey, driverId, truckId, driverName) VALUES (?, ?, ?, ?)"
+ configMethods:
+ - name: "bind"
+ args:
+ - reflist: ["FieldSelector-1", "FieldSelector-2", "FieldSelector-3", "FieldSelector-4"]
+```
+
+**N.B.:** References can only be used after (below) the object they point to has been declared.
+
+####Properties
+In addition to calling constructors with different arguments, Flux also allows you to configure components using
+JavaBean-like setter methods and fields declared as `public`:
+
+```yaml
+ - id: "spoutConfig"
+ className: "org.apache.storm.kafka.SpoutConfig"
+ constructorArgs:
+ # brokerHosts
+ - ref: "zkHosts"
+ # topic
+ - "myKafkaTopic"
+ # zkRoot
+ - "/kafkaSpout"
+ # id
+ - "myId"
+ properties:
+ - name: "ignoreZkOffsets"
+ value: true
+ - name: "scheme"
+ ref: "stringMultiScheme"
+```
+
+In the example above, the `properties` declaration will cause Flux to look for a public method in the `SpoutConfig` with
+the signature `setForceFromStart(boolean b)` and attempt to invoke it. If a setter method is not found, Flux will then
+look for a public instance variable with the name `ignoreZkOffsets` and attempt to set its value.
+
+References may also be used as property values.
+
+####Configuration Methods
+Conceptually, configuration methods are similar to Properties and Constructor Args -- they allow you to invoke an
+arbitrary method on an object after it is constructed. Configuration methods are useful for working with classes that
+don't expose JavaBean methods or have constructors that can fully configure the object. Common examples include classes
+that use the builder pattern for configuration/composition.
+
+The following YAML example creates a bolt and configures it by calling several methods:
+
+```yaml
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.flux.test.TestBolt"
+ parallelism: 1
+ configMethods:
+ - name: "withFoo"
+ args:
+ - "foo"
+ - name: "withNone"
+ # no args needed, so no "args" line
+ - name: "withBar"
+ args:
+ - "bar"
+ - name: "withFooBar"
+ args:
+ - "foo"
+ - "bar"
+```
+
+The signatures of the corresponding methods are as follows:
+
+```java
+ public void withFoo(String foo);
+ public void withNone(); // method with zero arguments
+ public void withBar(String bar);
+ public void withFooBar(String foo, String bar);
+```
+
+Arguments passed to configuration methods work much the same way as constructor arguments, and support references as
+well.
+
+### Using Java `enum`s in Contructor Arguments, References, Properties and Configuration Methods
+You can easily use Java `enum` values as arguments in a Flux YAML file, simply by referencing the name of the `enum`.
+
+For example, [Storm's HDFS module]() includes the following `enum` definition (simplified for brevity):
+
+```java
+public static enum Units {
+ KB, MB, GB, TB
+}
+```
+
+And the `org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy` class has the following constructor:
+
+```java
+public FileSizeRotationPolicy(float count, Units units)
+
+```
+The following Flux `component` definition could be used to call the constructor:
+
+```yaml
+ - id: "rotationPolicy"
+ className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
+ constructorArgs:
+ - 5.0
+ - MB
+```
+
+The above definition is functionally equivalent to the following Java code:
+
+```java
+// rotate files when they reach 5MB
+FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(5.0f, Units.MB);
+```
+
+## Topology Config
+The `config` section is simply a map of Storm topology configuration parameters that will be passed to the
+`org.apache.storm.StormSubmitter` as an instance of the `org.apache.storm.Config` class:
+
+```yaml
+config:
+ topology.workers: 4
+ topology.max.spout.pending: 1000
+ topology.message.timeout.secs: 30
+```
+
+# Existing Topologies
+If you have existing Storm topologies, you can still use Flux to deploy/run/test them. This feature allows you to
+leverage Flux Constructor Arguments, References, Properties, and Topology Config declarations for existing topology
+classes.
+
+The easiest way to use an existing topology class is to define
+a `getTopology()` instance method with one of the following signatures:
+
+```java
+public StormTopology getTopology(Map<String, Object> config)
+```
+or:
+
+```java
+public StormTopology getTopology(Config config)
+```
+
+You could then use the following YAML to configure your topology:
+
+```yaml
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopology"
+```
+
+If the class you would like to use as a topology source has a different method name (i.e. not `getTopology`), you can
+override it:
+
+```yaml
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopology"
+ methodName: "getTopologyWithDifferentMethodName"
+```
+
+__N.B.:__ The specified method must accept a single argument of type `java.util.Map<String, Object>` or
+`org.apache.storm.Config`, and return a `org.apache.storm.generated.StormTopology` object.
+
+# YAML DSL
+## Spouts and Bolts
+Spout and Bolts are configured in their own respective section of the YAML configuration. Spout and Bolt definitions
+are extensions to the `component` definition that add a `parallelism` parameter that sets the parallelism for a
+component when the topology is deployed.
+
+Because spout and bolt definitions extend `component` they support constructor arguments, references, and properties as
+well.
+
+Shell spout example:
+
+```yaml
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+```
+
+Kafka spout example:
+
+```yaml
+components:
+ - id: "stringScheme"
+ className: "org.apache.storm.kafka.StringScheme"
+
+ - id: "stringMultiScheme"
+ className: "org.apache.storm.spout.SchemeAsMultiScheme"
+ constructorArgs:
+ - ref: "stringScheme"
+
+ - id: "zkHosts"
+ className: "org.apache.storm.kafka.ZkHosts"
+ constructorArgs:
+ - "localhost:2181"
+
+# Alternative kafka config
+# - id: "kafkaConfig"
+# className: "org.apache.storm.kafka.KafkaConfig"
+# constructorArgs:
+# # brokerHosts
+# - ref: "zkHosts"
+# # topic
+# - "myKafkaTopic"
+# # clientId (optional)
+# - "myKafkaClientId"
+
+ - id: "spoutConfig"
+ className: "org.apache.storm.kafka.SpoutConfig"
+ constructorArgs:
+ # brokerHosts
+ - ref: "zkHosts"
+ # topic
+ - "myKafkaTopic"
+ # zkRoot
+ - "/kafkaSpout"
+ # id
+ - "myId"
+ properties:
+ - name: "ignoreZkOffsets"
+ value: true
+ - name: "scheme"
+ ref: "stringMultiScheme"
+
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "kafka-spout"
+ className: "org.apache.storm.kafka.KafkaSpout"
+ constructorArgs:
+ - ref: "spoutConfig"
+
+```
+
+Bolt Examples:
+
+```yaml
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+```
+## Streams and Stream Groupings
+Streams in Flux are represented as a list of connections (Graph edges, data flow, etc.) between the Spouts and Bolts in
+a topology, with an associated Grouping definition.
+
+A Stream definition has the following properties:
+
+**`name`:** A name for the connection (optional, currently unused)
+
+**`from`:** The `id` of a Spout or Bolt that is the source (publisher)
+
+**`to`:** The `id` of a Spout or Bolt that is the destination (subscriber)
+
+**`grouping`:** The stream grouping definition for the Stream
+
+A Grouping definition has the following properties:
+
+**`type`:** The type of grouping. One of `ALL`,`CUSTOM`,`DIRECT`,`SHUFFLE`,`LOCAL_OR_SHUFFLE`,`FIELDS`,`GLOBAL`, or `NONE`.
+
+**`streamId`:** The Storm stream ID (Optional. If unspecified will use the default stream)
+
+**`args`:** For the `FIELDS` grouping, a list of field names.
+
+**`customClass`** For the `CUSTOM` grouping, a definition of custom grouping class instance
+
+The `streams` definition example below sets up a topology with the following wiring:
+
+```
+ kafka-spout --> splitsentence --> count --> log
+```
+
+
+```yaml
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "kafka-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
+```
+
+### Custom Stream Groupings
+Custom stream groupings are defined by setting the grouping type to `CUSTOM` and defining a `customClass` parameter
+that tells Flux how to instantiate the custom class. The `customClass` definition extends `component`, so it supports
+constructor arguments, references, and properties as well.
+
+The example below creates a Stream with an instance of the `org.apache.storm.testing.NGrouping` custom stream grouping
+class.
+
+```yaml
+ - name: "bolt-1 --> bolt2"
+ from: "bolt-1"
+ to: "bolt-2"
+ grouping:
+ type: CUSTOM
+ customClass:
+ className: "org.apache.storm.testing.NGrouping"
+ constructorArgs:
+ - 1
+```
+
+## Includes and Overrides
+Flux allows you to include the contents of other YAML files, and have them treated as though they were defined in the
+same file. Includes may be either files, or classpath resources.
+
+Includes are specified as a list of maps:
+
+```yaml
+includes:
+ - resource: false
+ file: "src/test/resources/configs/shell_test.yaml"
+ override: false
+```
+
+If the `resource` property is set to `true`, the include will be loaded as a classpath resource from the value of the
+`file` attribute, otherwise it will be treated as a regular file.
+
+The `override` property controls how includes affect the values defined in the current file. If `override` is set to
+`true`, values in the included file will replace values in the current file being parsed. If `override` is set to
+`false`, values in the current file being parsed will take precedence, and the parser will refuse to replace them.
+
+**N.B.:** Includes are not yet recursive. Includes from included files will be ignored.
+
+
+## Basic Word Count Example
+
+This example uses a spout implemented in JavaScript, a bolt implemented in Python, and a bolt implemented in Java
+
+Topology YAML config:
+
+```yaml
+---
+name: "shell-topology"
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ parallelism: 1
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "sentence-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
+```
+
+
+## Micro-Batching (Trident) API Support
+Currenty, the Flux YAML DSL only supports the Core Storm API, but support for Storm's micro-batching API is planned.
+
+To use Flux with a Trident topology, define a topology getter method and reference it in your YAML config:
+
+```yaml
+name: "my-trident-topology"
+
+config:
+ topology.workers: 1
+
+topologySource:
+ className: "org.apache.storm.flux.test.TridentTopologySource"
+ # Flux will look for "getTopology", this will override that.
+ methodName: "getTopologyWithDifferentMethodName"
+```
+
+## Committer Sponsors
+
+ * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-core/pom.xml b/flux/flux-core/pom.xml
new file mode 100644
index 0000000..1cf4a9b
--- /dev/null
+++ b/flux/flux-core/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>flux-core</artifactId>
+ <packaging>jar</packaging>
+
+ <name>flux-core</name>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>flux-wrappers</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-kafka</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hdfs</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hbase</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.yaml</groupId>
+ <artifactId>snakeyaml</artifactId>
+ </dependency>
+ </dependencies>
+ <build>
+ <resources>
+ <resource>
+ <directory>src/main/resources</directory>
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <configuration>
+ <createDependencyReducedPom>true</createDependencyReducedPom>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.sf</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.dsa</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ <exclude>META-INF/*.rsa</exclude>
+ <exclude>META-INF/*.EC</exclude>
+ <exclude>META-INF/*.ec</exclude>
+ <exclude>META-INF/MSFTSIG.SF</exclude>
+ <exclude>META-INF/MSFTSIG.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <transformers>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+ <mainClass>org.apache.storm.flux.Flux</mainClass>
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java b/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
new file mode 100644
index 0000000..222bf2d
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.generated.TopologyInitialStatus;
+import org.apache.storm.utils.Utils;
+import org.apache.commons.cli.*;
+import org.apache.storm.flux.model.*;
+import org.apache.storm.flux.parser.FluxParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+
+/**
+ * Flux entry point.
+ *
+ */
+public class Flux {
+ private static final Logger LOG = LoggerFactory.getLogger(Flux.class);
+
+ private static final Long DEFAULT_LOCAL_SLEEP_TIME = 60000l;
+
+ private static final Long DEFAULT_ZK_PORT = 2181l;
+
+ private static final String OPTION_LOCAL = "local";
+ private static final String OPTION_REMOTE = "remote";
+ private static final String OPTION_RESOURCE = "resource";
+ private static final String OPTION_SLEEP = "sleep";
+ private static final String OPTION_DRY_RUN = "dry-run";
+ private static final String OPTION_NO_DETAIL = "no-detail";
+ private static final String OPTION_NO_SPLASH = "no-splash";
+ private static final String OPTION_INACTIVE = "inactive";
+ private static final String OPTION_ZOOKEEPER = "zookeeper";
+ private static final String OPTION_FILTER = "filter";
+ private static final String OPTION_ENV_FILTER = "env-filter";
+
+ public static void main(String[] args) throws Exception {
+ Options options = new Options();
+
+ options.addOption(option(0, "l", OPTION_LOCAL, "Run the topology in local mode."));
+
+ options.addOption(option(0, "r", OPTION_REMOTE, "Deploy the topology to a remote cluster."));
+
+ options.addOption(option(0, "R", OPTION_RESOURCE, "Treat the supplied path as a classpath resource instead of a file."));
+
+ options.addOption(option(1, "s", OPTION_SLEEP, "ms", "When running locally, the amount of time to sleep (in ms.) " +
+ "before killing the topology and shutting down the local cluster."));
+
+ options.addOption(option(0, "d", OPTION_DRY_RUN, "Do not run or deploy the topology. Just build, validate, " +
+ "and print information about the topology."));
+
+ options.addOption(option(0, "q", OPTION_NO_DETAIL, "Suppress the printing of topology details."));
+
+ options.addOption(option(0, "n", OPTION_NO_SPLASH, "Suppress the printing of the splash screen."));
+
+ options.addOption(option(0, "i", OPTION_INACTIVE, "Deploy the topology, but do not activate it."));
+
+ options.addOption(option(1, "z", OPTION_ZOOKEEPER, "host:port", "When running in local mode, use the ZooKeeper at the " +
+ "specified <host>:<port> instead of the in-process ZooKeeper. (requires Storm 0.9.3 or later)"));
+
+ options.addOption(option(1, "f", OPTION_FILTER, "file", "Perform property substitution. Use the specified file " +
+ "as a source of properties, and replace keys identified with {$[property name]} with the value defined " +
+ "in the properties file."));
+
+ options.addOption(option(0, "e", OPTION_ENV_FILTER, "Perform environment variable substitution. Replace keys" +
+ "identified with `${ENV-[NAME]}` will be replaced with the corresponding `NAME` environment value"));
+
+ CommandLineParser parser = new BasicParser();
+ CommandLine cmd = parser.parse(options, args);
+
+ if (cmd.getArgs().length != 1) {
+ usage(options);
+ System.exit(1);
+ }
+ runCli(cmd);
+ }
+
+ private static Option option(int argCount, String shortName, String longName, String description){
+ return option(argCount, shortName, longName, longName, description);
+ }
+
+ private static Option option(int argCount, String shortName, String longName, String argName, String description){
+ Option option = OptionBuilder.hasArgs(argCount)
+ .withArgName(argName)
+ .withLongOpt(longName)
+ .withDescription(description)
+ .create(shortName);
+ return option;
+ }
+
+ private static void usage(Options options) {
+ HelpFormatter formatter = new HelpFormatter();
+ formatter.printHelp("storm jar <my_topology_uber_jar.jar> " +
+ Flux.class.getName() +
+ " [options] <topology-config.yaml>", options);
+ }
+
+ private static void runCli(CommandLine cmd)throws Exception {
+ if(!cmd.hasOption(OPTION_NO_SPLASH)) {
+ printSplash();
+ }
+
+ boolean dumpYaml = cmd.hasOption("dump-yaml");
+
+ TopologyDef topologyDef = null;
+ String filePath = (String)cmd.getArgList().get(0);
+
+ // TODO conditionally load properties from a file our resource
+ String filterProps = null;
+ if(cmd.hasOption(OPTION_FILTER)){
+ filterProps = cmd.getOptionValue(OPTION_FILTER);
+ }
+
+
+ boolean envFilter = cmd.hasOption(OPTION_ENV_FILTER);
+ if(cmd.hasOption(OPTION_RESOURCE)){
+ printf("Parsing classpath resource: %s", filePath);
+ topologyDef = FluxParser.parseResource(filePath, dumpYaml, true, filterProps, envFilter);
+ } else {
+ printf("Parsing file: %s",
+ new File(filePath).getAbsolutePath());
+ topologyDef = FluxParser.parseFile(filePath, dumpYaml, true, filterProps, envFilter);
+ }
+
+
+ String topologyName = topologyDef.getName();
+ // merge contents of `config` into topology config
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+
+ if(!cmd.hasOption(OPTION_NO_DETAIL)){
+ printTopologyInfo(context);
+ }
+
+ if(!cmd.hasOption(OPTION_DRY_RUN)) {
+ if (cmd.hasOption(OPTION_REMOTE)) {
+ LOG.info("Running remotely...");
+ // should the topology be active or inactive
+ SubmitOptions submitOptions = null;
+ if(cmd.hasOption(OPTION_INACTIVE)){
+ LOG.info("Deploying topology in an INACTIVE state...");
+ submitOptions = new SubmitOptions(TopologyInitialStatus.INACTIVE);
+ } else {
+ LOG.info("Deploying topology in an ACTIVE state...");
+ submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+ }
+ StormSubmitter.submitTopology(topologyName, conf, topology, submitOptions, null);
+ } else {
+ LOG.info("Running in local mode...");
+
+ String sleepStr = cmd.getOptionValue(OPTION_SLEEP);
+ Long sleepTime = DEFAULT_LOCAL_SLEEP_TIME;
+ if (sleepStr != null) {
+ sleepTime = Long.parseLong(sleepStr);
+ }
+ LOG.debug("Sleep time: {}", sleepTime);
+ LocalCluster cluster = null;
+
+ // in-process or external zookeeper
+ if(cmd.hasOption(OPTION_ZOOKEEPER)){
+ String zkStr = cmd.getOptionValue(OPTION_ZOOKEEPER);
+ LOG.info("Using ZooKeeper at '{}' instead of in-process one.", zkStr);
+ long zkPort = DEFAULT_ZK_PORT;
+ String zkHost = null;
+ if(zkStr.contains(":")){
+ String[] hostPort = zkStr.split(":");
+ zkHost = hostPort[0];
+ zkPort = hostPort.length > 1 ? Long.parseLong(hostPort[1]) : DEFAULT_ZK_PORT;
+
+ } else {
+ zkHost = zkStr;
+ }
+ // the following constructor is only available in 0.9.3 and later
+ try {
+ cluster = new LocalCluster(zkHost, zkPort);
+ } catch (NoSuchMethodError e){
+ LOG.error("The --zookeeper option can only be used with Apache Storm 0.9.3 and later.");
+ System.exit(1);
+ }
+ } else {
+ cluster = new LocalCluster();
+ }
+ try (LocalTopology topo = cluster.submitTopology(topologyName, conf, topology)) {
+ Utils.sleep(sleepTime);
+ } finally {
+ cluster.shutdown();
+ }
+ }
+ }
+ }
+
+ static void printTopologyInfo(ExecutionContext ctx){
+ TopologyDef t = ctx.getTopologyDef();
+ if(t.isDslTopology()) {
+ print("---------- TOPOLOGY DETAILS ----------");
+
+ printf("Topology Name: %s", t.getName());
+ print("--------------- SPOUTS ---------------");
+ for (SpoutDef s : t.getSpouts()) {
+ printf("%s [%d] (%s)", s.getId(), s.getParallelism(), s.getClassName());
+ }
+ print("---------------- BOLTS ---------------");
+ for (BoltDef b : t.getBolts()) {
+ printf("%s [%d] (%s)", b.getId(), b.getParallelism(), b.getClassName());
+ }
+
+ print("--------------- STREAMS ---------------");
+ for (StreamDef sd : t.getStreams()) {
+ printf("%s --%s--> %s", sd.getFrom(), sd.getGrouping().getType(), sd.getTo());
+ }
+ print("--------------------------------------");
+ }
+ }
+
+ // save a little typing
+ private static void printf(String format, Object... args){
+ print(String.format(format, args));
+ }
+
+ private static void print(String string){
+ System.out.println(string);
+ }
+
+ private static void printSplash() throws IOException {
+ // banner
+ InputStream is = Flux.class.getResourceAsStream("/splash.txt");
+ if(is != null){
+ InputStreamReader isr = new InputStreamReader(is, "UTF-8");
+ BufferedReader br = new BufferedReader(isr);
+ String line = null;
+ while((line = br.readLine()) != null){
+ System.out.println(line);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java b/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
new file mode 100644
index 0000000..e79dfb7
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
@@ -0,0 +1,630 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.topology.*;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.flux.model.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.*;
+import java.util.*;
+
+public class FluxBuilder {
+ private static Logger LOG = LoggerFactory.getLogger(FluxBuilder.class);
+
+ /**
+ * Given a topology definition, return a populated `org.apache.storm.Config` instance.
+ *
+ * @param topologyDef
+ * @return
+ */
+ public static Config buildConfig(TopologyDef topologyDef) {
+ // merge contents of `config` into topology config
+ Config conf = new Config();
+ conf.putAll(topologyDef.getConfig());
+ return conf;
+ }
+
+ /**
+ * Given a topology definition, return a Storm topology that can be run either locally or remotely.
+ *
+ * @param context
+ * @return
+ * @throws IllegalAccessException
+ * @throws InstantiationException
+ * @throws ClassNotFoundException
+ * @throws NoSuchMethodException
+ * @throws InvocationTargetException
+ */
+ public static StormTopology buildTopology(ExecutionContext context) throws IllegalAccessException,
+ InstantiationException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+
+ StormTopology topology = null;
+ TopologyDef topologyDef = context.getTopologyDef();
+
+ if(!topologyDef.validate()){
+ throw new IllegalArgumentException("Invalid topology config. Spouts, bolts and streams cannot be " +
+ "defined in the same configuration as a topologySource.");
+ }
+
+ // build components that may be referenced by spouts, bolts, etc.
+ // the map will be a String --> Object where the object is a fully
+ // constructed class instance
+ buildComponents(context);
+
+ if(topologyDef.isDslTopology()) {
+ // This is a DSL (YAML, etc.) topology...
+ LOG.info("Detected DSL topology...");
+
+ TopologyBuilder builder = new TopologyBuilder();
+
+ // create spouts
+ buildSpouts(context, builder);
+
+ // we need to be able to lookup bolts by id, then switch based
+ // on whether they are IBasicBolt or IRichBolt instances
+ buildBolts(context);
+
+ // process stream definitions
+ buildStreamDefinitions(context, builder);
+
+ topology = builder.createTopology();
+ } else {
+ // user class supplied...
+ // this also provides a bridge to Trident...
+ LOG.info("A topology source has been specified...");
+ ObjectDef def = topologyDef.getTopologySource();
+ topology = buildExternalTopology(def, context);
+ }
+ return topology;
+ }
+
+ /**
+ * Given a `java.lang.Object` instance and a method name, attempt to find a method that matches the input
+ * parameter: `java.util.Map` or `org.apache.storm.Config`.
+ *
+ * @param topologySource object to inspect for the specified method
+ * @param methodName name of the method to look for
+ * @return
+ * @throws NoSuchMethodException
+ */
+ private static Method findGetTopologyMethod(Object topologySource, String methodName) throws NoSuchMethodException {
+ Class clazz = topologySource.getClass();
+ Method[] methods = clazz.getMethods();
+ ArrayList<Method> candidates = new ArrayList<Method>();
+ for(Method method : methods){
+ if(!method.getName().equals(methodName)){
+ continue;
+ }
+ if(!method.getReturnType().equals(StormTopology.class)){
+ continue;
+ }
+ Class[] paramTypes = method.getParameterTypes();
+ if(paramTypes.length != 1){
+ continue;
+ }
+ if(paramTypes[0].isAssignableFrom(Map.class) || paramTypes[0].isAssignableFrom(Config.class)){
+ candidates.add(method);
+ }
+ }
+
+ if(candidates.size() == 0){
+ throw new IllegalArgumentException("Unable to find method '" + methodName + "' method in class: " + clazz.getName());
+ } else if (candidates.size() > 1){
+ LOG.warn("Found multiple candidate methods in class '" + clazz.getName() + "'. Using the first one found");
+ }
+
+ return candidates.get(0);
+ }
+
+ /**
+ * @param context
+ * @param builder
+ */
+ private static void buildStreamDefinitions(ExecutionContext context, TopologyBuilder builder)
+ throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException,
+ IllegalAccessException, NoSuchFieldException {
+ TopologyDef topologyDef = context.getTopologyDef();
+ // process stream definitions
+ HashMap<String, BoltDeclarer> declarers = new HashMap<String, BoltDeclarer>();
+ for (StreamDef stream : topologyDef.getStreams()) {
+ Object boltObj = context.getBolt(stream.getTo());
+ BoltDeclarer declarer = declarers.get(stream.getTo());
+ if (boltObj instanceof IRichBolt) {
+ if(declarer == null) {
+ declarer = builder.setBolt(stream.getTo(),
+ (IRichBolt) boltObj,
+ topologyDef.parallelismForBolt(stream.getTo()));
+ declarers.put(stream.getTo(), declarer);
+ }
+ } else if (boltObj instanceof IBasicBolt) {
+ if(declarer == null) {
+ declarer = builder.setBolt(
+ stream.getTo(),
+ (IBasicBolt) boltObj,
+ topologyDef.parallelismForBolt(stream.getTo()));
+ declarers.put(stream.getTo(), declarer);
+ }
+ } else if (boltObj instanceof IWindowedBolt) {
+ if(declarer == null) {
+ declarer = builder.setBolt(
+ stream.getTo(),
+ (IWindowedBolt) boltObj,
+ topologyDef.parallelismForBolt(stream.getTo()));
+ declarers.put(stream.getTo(), declarer);
+ }
+ } else if (boltObj instanceof IStatefulBolt) {
+ if(declarer == null) {
+ declarer = builder.setBolt(
+ stream.getTo(),
+ (IStatefulBolt) boltObj,
+ topologyDef.parallelismForBolt(stream.getTo()));
+ declarers.put(stream.getTo(), declarer);
+ }
+ } else {
+ throw new IllegalArgumentException("Class does not appear to be a bolt: " +
+ boltObj.getClass().getName());
+ }
+
+ GroupingDef grouping = stream.getGrouping();
+ // if the streamId is defined, use it for the grouping, otherwise assume storm's default stream
+ String streamId = (grouping.getStreamId() == null ? Utils.DEFAULT_STREAM_ID : grouping.getStreamId());
+
+
+ switch (grouping.getType()) {
+ case SHUFFLE:
+ declarer.shuffleGrouping(stream.getFrom(), streamId);
+ break;
+ case FIELDS:
+ //TODO check for null grouping args
+ declarer.fieldsGrouping(stream.getFrom(), streamId, new Fields(grouping.getArgs()));
+ break;
+ case ALL:
+ declarer.allGrouping(stream.getFrom(), streamId);
+ break;
+ case DIRECT:
+ declarer.directGrouping(stream.getFrom(), streamId);
+ break;
+ case GLOBAL:
+ declarer.globalGrouping(stream.getFrom(), streamId);
+ break;
+ case LOCAL_OR_SHUFFLE:
+ declarer.localOrShuffleGrouping(stream.getFrom(), streamId);
+ break;
+ case NONE:
+ declarer.noneGrouping(stream.getFrom(), streamId);
+ break;
+ case CUSTOM:
+ declarer.customGrouping(stream.getFrom(), streamId,
+ buildCustomStreamGrouping(stream.getGrouping().getCustomClass(), context));
+ break;
+ default:
+ throw new UnsupportedOperationException("unsupported grouping type: " + grouping);
+ }
+ }
+ }
+
+ private static void applyProperties(ObjectDef bean, Object instance, ExecutionContext context) throws
+ IllegalAccessException, InvocationTargetException, NoSuchFieldException {
+ List<PropertyDef> props = bean.getProperties();
+ Class clazz = instance.getClass();
+ if (props != null) {
+ for (PropertyDef prop : props) {
+ Object value = prop.isReference() ? context.getComponent(prop.getRef()) : prop.getValue();
+ Method setter = findSetter(clazz, prop.getName(), value);
+ if (setter != null) {
+ LOG.debug("found setter, attempting to invoke");
+ // invoke setter
+ setter.invoke(instance, new Object[]{value});
+ } else {
+ // look for a public instance variable
+ LOG.debug("no setter found. Looking for a public instance variable...");
+ Field field = findPublicField(clazz, prop.getName(), value);
+ if (field != null) {
+ field.set(instance, value);
+ }
+ }
+ }
+ }
+ }
+
+ private static Field findPublicField(Class clazz, String property, Object arg) throws NoSuchFieldException {
+ Field field = clazz.getField(property);
+ return field;
+ }
+
+ private static Method findSetter(Class clazz, String property, Object arg) {
+ String setterName = toSetterName(property);
+ Method retval = null;
+ Method[] methods = clazz.getMethods();
+ for (Method method : methods) {
+ if (setterName.equals(method.getName())) {
+ LOG.debug("Found setter method: " + method.getName());
+ retval = method;
+ }
+ }
+ return retval;
+ }
+
+ private static String toSetterName(String name) {
+ return "set" + name.substring(0, 1).toUpperCase() + name.substring(1, name.length());
+ }
+
+ private static List<Object> resolveReferences(List<Object> args, ExecutionContext context) {
+ LOG.debug("Checking arguments for references.");
+ List<Object> cArgs = new ArrayList<Object>();
+ // resolve references
+ for (Object arg : args) {
+ if (arg instanceof BeanReference) {
+ cArgs.add(context.getComponent(((BeanReference) arg).getId()));
+ } else if (arg instanceof BeanListReference) {
+ List<Object> components = new ArrayList<>();
+ BeanListReference ref = (BeanListReference) arg;
+ for (String id : ref.getIds()) {
+ components.add(context.getComponent(id));
+ }
+
+ LOG.debug("BeanListReference resolved as {}", components);
+ cArgs.add(components);
+ } else {
+ cArgs.add(arg);
+ }
+ }
+ return cArgs;
+ }
+
+ private static Object buildObject(ObjectDef def, ExecutionContext context) throws ClassNotFoundException,
+ IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+ Class clazz = Class.forName(def.getClassName());
+ Object obj = null;
+ if (def.hasConstructorArgs()) {
+ LOG.debug("Found constructor arguments in definition: " + def.getConstructorArgs().getClass().getName());
+ List<Object> cArgs = def.getConstructorArgs();
+ if(def.hasReferences()){
+ cArgs = resolveReferences(cArgs, context);
+ }
+ Constructor con = findCompatibleConstructor(cArgs, clazz);
+ if (con != null) {
+ LOG.debug("Found something seemingly compatible, attempting invocation...");
+ obj = con.newInstance(getArgsWithListCoercian(cArgs, con.getParameterTypes()));
+ } else {
+ String msg = String.format("Couldn't find a suitable constructor for class '%s' with arguments '%s'.",
+ clazz.getName(),
+ cArgs);
+ throw new IllegalArgumentException(msg);
+ }
+ } else {
+ obj = clazz.newInstance();
+ }
+ applyProperties(def, obj, context);
+ invokeConfigMethods(def, obj, context);
+ return obj;
+ }
+
+ private static StormTopology buildExternalTopology(ObjectDef def, ExecutionContext context)
+ throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException,
+ InvocationTargetException, NoSuchFieldException {
+
+ Object topologySource = buildObject(def, context);
+
+ String methodName = context.getTopologyDef().getTopologySource().getMethodName();
+ Method getTopology = findGetTopologyMethod(topologySource, methodName);
+ if(getTopology.getParameterTypes()[0].equals(Config.class)){
+ Config config = new Config();
+ config.putAll(context.getTopologyDef().getConfig());
+ return (StormTopology) getTopology.invoke(topologySource, config);
+ } else {
+ return (StormTopology) getTopology.invoke(topologySource, context.getTopologyDef().getConfig());
+ }
+ }
+
+ private static CustomStreamGrouping buildCustomStreamGrouping(ObjectDef def, ExecutionContext context)
+ throws ClassNotFoundException,
+ IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+ Object grouping = buildObject(def, context);
+ return (CustomStreamGrouping)grouping;
+ }
+
+ /**
+ * Given a topology definition, resolve and instantiate all components found and return a map
+ * keyed by the component id.
+ */
+ private static void buildComponents(ExecutionContext context) throws ClassNotFoundException, NoSuchMethodException,
+ IllegalAccessException, InvocationTargetException, InstantiationException, NoSuchFieldException {
+ Collection<BeanDef> cDefs = context.getTopologyDef().getComponents();
+ if (cDefs != null) {
+ for (BeanDef bean : cDefs) {
+ Object obj = buildObject(bean, context);
+ context.addComponent(bean.getId(), obj);
+ }
+ }
+ }
+
+
+ private static void buildSpouts(ExecutionContext context, TopologyBuilder builder) throws ClassNotFoundException,
+ NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, NoSuchFieldException {
+ for (SpoutDef sd : context.getTopologyDef().getSpouts()) {
+ IRichSpout spout = buildSpout(sd, context);
+ builder.setSpout(sd.getId(), spout, sd.getParallelism());
+ context.addSpout(sd.getId(), spout);
+ }
+ }
+
+ /**
+ * Given a spout definition, return a Storm spout implementation by attempting to find a matching constructor
+ * in the given spout class. Perform list to array conversion as necessary.
+ */
+ private static IRichSpout buildSpout(SpoutDef def, ExecutionContext context) throws ClassNotFoundException,
+ IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+ return (IRichSpout)buildObject(def, context);
+ }
+
+ /**
+ * Given a list of bolt definitions, build a map of Storm bolts with the bolt definition id as the key.
+ * Attempt to coerce the given constructor arguments to a matching bolt constructor as much as possible.
+ */
+ private static void buildBolts(ExecutionContext context) throws ClassNotFoundException, IllegalAccessException,
+ InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+ for (BoltDef def : context.getTopologyDef().getBolts()) {
+ Class clazz = Class.forName(def.getClassName());
+ Object bolt = buildObject(def, context);
+ context.addBolt(def.getId(), bolt);
+ }
+ }
+
+ /**
+ * Given a list of constructor arguments, and a target class, attempt to find a suitable constructor.
+ *
+ */
+ private static Constructor findCompatibleConstructor(List<Object> args, Class target) throws NoSuchMethodException {
+ Constructor retval = null;
+ int eligibleCount = 0;
+
+ LOG.debug("Target class: {}, constructor args: {}", target.getName(), args);
+ Constructor[] cons = target.getDeclaredConstructors();
+
+ for (Constructor con : cons) {
+ Class[] paramClasses = con.getParameterTypes();
+ if (paramClasses.length == args.size()) {
+ LOG.debug("found constructor with same number of args..");
+ boolean invokable = canInvokeWithArgs(args, con.getParameterTypes());
+ if (invokable) {
+ retval = con;
+ eligibleCount++;
+ }
+ LOG.debug("** invokable --> {}", invokable);
+ } else {
+ LOG.debug("Skipping constructor with wrong number of arguments.");
+ }
+ }
+ if (eligibleCount > 1) {
+ LOG.warn("Found multiple invokable constructors for class {}, given arguments {}. Using the last one found.",
+ target, args);
+ }
+ return retval;
+ }
+
+
+ public static void invokeConfigMethods(ObjectDef bean, Object instance, ExecutionContext context)
+ throws InvocationTargetException, IllegalAccessException {
+
+ List<ConfigMethodDef> methodDefs = bean.getConfigMethods();
+ if(methodDefs == null || methodDefs.size() == 0){
+ return;
+ }
+ Class clazz = instance.getClass();
+ for(ConfigMethodDef methodDef : methodDefs){
+ List<Object> args = methodDef.getArgs();
+ if (args == null){
+ args = new ArrayList();
+ }
+ if(methodDef.hasReferences()){
+ args = resolveReferences(args, context);
+ }
+ String methodName = methodDef.getName();
+ Method method = findCompatibleMethod(args, clazz, methodName);
+ if(method != null) {
+ Object[] methodArgs = getArgsWithListCoercian(args, method.getParameterTypes());
+ method.invoke(instance, methodArgs);
+ } else {
+ String msg = String.format("Unable to find configuration method '%s' in class '%s' with arguments %s.",
+ new Object[]{methodName, clazz.getName(), args});
+ throw new IllegalArgumentException(msg);
+ }
+ }
+ }
+
+ private static Method findCompatibleMethod(List<Object> args, Class target, String methodName){
+ Method retval = null;
+ int eligibleCount = 0;
+
+ LOG.debug("Target class: {}, methodName: {}, args: {}", target.getName(), methodName, args);
+ Method[] methods = target.getMethods();
+
+ for (Method method : methods) {
+ Class[] paramClasses = method.getParameterTypes();
+ if (paramClasses.length == args.size() && method.getName().equals(methodName)) {
+ LOG.debug("found constructor with same number of args..");
+ boolean invokable = false;
+ if (args.size() == 0){
+ // it's a method with zero args
+ invokable = true;
+ } else {
+ invokable = canInvokeWithArgs(args, method.getParameterTypes());
+ }
+ if (invokable) {
+ retval = method;
+ eligibleCount++;
+ }
+ LOG.debug("** invokable --> {}", invokable);
+ } else {
+ LOG.debug("Skipping method with wrong number of arguments.");
+ }
+ }
+ if (eligibleCount > 1) {
+ LOG.warn("Found multiple invokable methods for class {}, method {}, given arguments {}. " +
+ "Using the last one found.",
+ new Object[]{target, methodName, args});
+ }
+ return retval;
+ }
+
+ /**
+ * Given a java.util.List of contructor/method arguments, and a list of parameter types, attempt to convert the
+ * list to an java.lang.Object array that can be used to invoke the constructor. If an argument needs
+ * to be coerced from a List to an Array, do so.
+ */
+ private static Object[] getArgsWithListCoercian(List<Object> args, Class[] parameterTypes) {
+// Class[] parameterTypes = constructor.getParameterTypes();
+ if (parameterTypes.length != args.size()) {
+ throw new IllegalArgumentException("Contructor parameter count does not egual argument size.");
+ }
+ Object[] constructorParams = new Object[args.size()];
+
+ // loop through the arguments, if we hit a list that has to be convered to an array,
+ // perform the conversion
+ for (int i = 0; i < args.size(); i++) {
+ Object obj = args.get(i);
+ Class paramType = parameterTypes[i];
+ Class objectType = obj.getClass();
+ LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
+ paramType, objectType);
+ if (paramType.equals(objectType)) {
+ LOG.debug("They are the same class.");
+ constructorParams[i] = args.get(i);
+ continue;
+ }
+ if (paramType.isAssignableFrom(objectType)) {
+ LOG.debug("Assignment is possible.");
+ constructorParams[i] = args.get(i);
+ continue;
+ }
+ if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+ LOG.debug("Its a primitive boolean.");
+ Boolean bool = (Boolean)args.get(i);
+ constructorParams[i] = bool.booleanValue();
+ continue;
+ }
+ if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
+ LOG.debug("Its a primitive number.");
+ Number num = (Number)args.get(i);
+ if(paramType == Float.TYPE){
+ constructorParams[i] = num.floatValue();
+ } else if (paramType == Double.TYPE) {
+ constructorParams[i] = num.doubleValue();
+ } else if (paramType == Long.TYPE) {
+ constructorParams[i] = num.longValue();
+ } else if (paramType == Integer.TYPE) {
+ constructorParams[i] = num.intValue();
+ } else if (paramType == Short.TYPE) {
+ constructorParams[i] = num.shortValue();
+ } else if (paramType == Byte.TYPE) {
+ constructorParams[i] = num.byteValue();
+ } else {
+ constructorParams[i] = args.get(i);
+ }
+ continue;
+ }
+
+ // enum conversion
+ if(paramType.isEnum() && objectType.equals(String.class)){
+ LOG.debug("Yes, will convert a String to enum");
+ constructorParams[i] = Enum.valueOf(paramType, (String)args.get(i));
+ continue;
+ }
+
+ // List to array conversion
+ if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
+ // TODO more collection content type checking
+ LOG.debug("Conversion appears possible...");
+ List list = (List) obj;
+ LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), list.get(0).getClass());
+
+ // create an array of the right type
+ Object newArrayObj = Array.newInstance(paramType.getComponentType(), list.size());
+ for (int j = 0; j < list.size(); j++) {
+ Array.set(newArrayObj, j, list.get(j));
+
+ }
+ constructorParams[i] = newArrayObj;
+ LOG.debug("After conversion: {}", constructorParams[i]);
+ }
+ }
+ return constructorParams;
+ }
+
+
+ /**
+ * Determine if the given constructor/method parameter types are compatible given arguments List. Consider if
+ * list coercian can make it possible.
+ *
+ * @param args
+ * @param parameterTypes
+ * @return
+ */
+ private static boolean canInvokeWithArgs(List<Object> args, Class[] parameterTypes) {
+ if (parameterTypes.length != args.size()) {
+ LOG.warn("parameter types were the wrong size");
+ return false;
+ }
+
+ for (int i = 0; i < args.size(); i++) {
+ Object obj = args.get(i);
+ if (obj == null) {
+ throw new IllegalArgumentException("argument shouldn't be null - index: " + i);
+ }
+ Class paramType = parameterTypes[i];
+ Class objectType = obj.getClass();
+ LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
+ paramType, objectType);
+ if (paramType.equals(objectType)) {
+ LOG.debug("Yes, they are the same class.");
+ } else if (paramType.isAssignableFrom(objectType)) {
+ LOG.debug("Yes, assignment is possible.");
+ } else if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+ LOG.debug("Yes, assignment is possible.");
+ } else if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
+ LOG.debug("Yes, assignment is possible.");
+ } else if(paramType.isEnum() && objectType.equals(String.class)){
+ LOG.debug("Yes, will convert a String to enum");
+ } else if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
+ // TODO more collection content type checking
+ LOG.debug("Assignment is possible if we convert a List to an array.");
+ LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), ((List) obj).get(0).getClass());
+ } else {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public static boolean isPrimitiveNumber(Class clazz){
+ return clazz.isPrimitive() && !clazz.equals(boolean.class);
+ }
+
+ public static boolean isPrimitiveBoolean(Class clazz){
+ return clazz.isPrimitive() && clazz.equals(boolean.class);
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java b/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
new file mode 100644
index 0000000..2777854
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.api;
+
+
+import org.apache.storm.generated.StormTopology;
+
+import java.util.Map;
+
+/**
+ * Marker interface for objects that can produce `StormTopology` objects.
+ *
+ * If a `topology-source` class implements the `getTopology()` method, Flux will
+ * call that method. Otherwise, it will introspect the given class and look for a
+ * similar method that produces a `StormTopology` instance.
+ *
+ * Note that it is not strictly necessary for a class to implement this interface.
+ * If a class defines a method with a similar signature, Flux should be able to find
+ * and invoke it.
+ *
+ */
+public interface TopologySource {
+ public StormTopology getTopology(Map<String, Object> config);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
new file mode 100644
index 0000000..f0247ed
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * A representation of a Java object that is uniquely identifyable, and given a className, constructor arguments,
+ * and properties, can be instantiated.
+ */
+public class BeanDef extends ObjectDef {
+ private String id;
+
+ public String getId() {
+ return id;
+ }
+
+ public void setId(String id) {
+ this.id = id;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
new file mode 100644
index 0000000..652210c
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import java.util.List;
+
+/**
+ * A bean list reference is a list of bean reference.
+ */
+public class BeanListReference {
+ public List<String> ids;
+
+ public BeanListReference(){}
+
+ public BeanListReference(List<String> ids){
+ this.ids = ids;
+ }
+
+ public List<String> getIds() {
+ return ids;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
new file mode 100644
index 0000000..bd236f1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * A bean reference is simply a string pointer to another id.
+ */
+public class BeanReference {
+ public String id;
+
+ public BeanReference(){}
+
+ public BeanReference(String id){
+ this.id = id;
+ }
+
+ public String getId() {
+ return id;
+ }
+
+ public void setId(String id) {
+ this.id = id;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
new file mode 100644
index 0000000..362abf1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * Bean representation of a Storm bolt.
+ */
+public class BoltDef extends VertexDef {
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
new file mode 100644
index 0000000..69cabc3
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ConfigMethodDef {
+ private String name;
+ private List<Object> args;
+ private boolean hasReferences = false;
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public List<Object> getArgs() {
+ return args;
+ }
+
+ public void setArgs(List<Object> args) {
+
+ List<Object> newVal = new ArrayList<Object>();
+ for(Object obj : args){
+ if(obj instanceof LinkedHashMap){
+ Map map = (Map)obj;
+ if(map.containsKey("ref") && map.size() == 1){
+ newVal.add(new BeanReference((String)map.get("ref")));
+ this.hasReferences = true;
+ } else if (map.containsKey("reflist") && map.size() == 1) {
+ newVal.add(new BeanListReference((List<String>) map.get("reflist")));
+ this.hasReferences = true;
+ } else {
+ newVal.add(obj);
+ }
+ } else {
+ newVal.add(obj);
+ }
+ }
+ this.args = newVal;
+ }
+
+ public boolean hasReferences(){
+ return this.hasReferences;
+ }
+}
[16/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
deleted file mode 100644
index 97995c7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.calcite.rel.stream.Delta;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.AggregateFunction;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Compile RelNodes into individual functions.
- */
-class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
- public static Joiner NEW_LINE_JOINER = Joiner.on('\n');
-
- private final PrintWriter pw;
- private final JavaTypeFactory typeFactory;
- private final RexNodeToJavaCodeCompiler rexCompiler;
-
- private static final String STAGE_PROLOGUE = NEW_LINE_JOINER.join(
- " private static final ChannelHandler %1$s = ",
- " new AbstractChannelHandler() {",
- " @Override",
- " public void dataReceived(ChannelContext ctx, Values _data) {",
- ""
- );
-
- private static final String AGGREGATE_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
- " private static final ChannelHandler %1$s = ",
- " new AbstractChannelHandler() {",
- " private final Values EMPTY_VALUES = new Values();",
- " private final Map<List<Object>, Map<String, Object>> state = new LinkedHashMap<>();",
- " private final int[] groupIndices = new int[] {%2$s};",
- " private List<Object> getGroupValues(Values _data) {",
- " List<Object> res = new ArrayList<>();",
- " for (int i: groupIndices) {",
- " res.add(_data.get(i));",
- " }",
- " return res;",
- " }",
- "",
- " @Override",
- " public void flush(ChannelContext ctx) {",
- " emitAggregateResults(ctx);",
- " super.flush(ctx);",
- " state.clear();",
- " }",
- "",
- " private void emitAggregateResults(ChannelContext ctx) {",
- " for (Map.Entry<List<Object>, Map<String, Object>> entry: state.entrySet()) {",
- " List<Object> groupValues = entry.getKey();",
- " Map<String, Object> accumulators = entry.getValue();",
- " %3$s",
- " }",
- " }",
- "",
- " @Override",
- " public void dataReceived(ChannelContext ctx, Values _data) {",
- ""
- );
-
- private static final String JOIN_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
- " private static final ChannelHandler %1$s = ",
- " new AbstractChannelHandler() {",
- " Object left = %2$s;",
- " Object right = %3$s;",
- " Object source = null;",
- " List<Values> leftRows = new ArrayList<>();",
- " List<Values> rightRows = new ArrayList<>();",
- " boolean leftDone = false;",
- " boolean rightDone = false;",
- " int[] ordinals = new int[] {%4$s, %5$s};",
- "",
- " Multimap<Object, Values> getJoinTable(List<Values> rows, int joinIndex) {",
- " Multimap<Object, Values> m = ArrayListMultimap.create();",
- " for(Values v: rows) {",
- " m.put(v.get(joinIndex), v);",
- " }",
- " return m;",
- " }",
- "",
- " List<Values> join(Multimap<Object, Values> tab, List<Values> rows, int rowIdx, boolean rev) {",
- " List<Values> res = new ArrayList<>();",
- " for (Values row: rows) {",
- " for (Values mapValue: tab.get(row.get(rowIdx))) {",
- " if (mapValue != null) {",
- " Values joinedRow = new Values();",
- " if(rev) {",
- " joinedRow.addAll(row);",
- " joinedRow.addAll(mapValue);",
- " } else {",
- " joinedRow.addAll(mapValue);",
- " joinedRow.addAll(row);",
- " }",
- " res.add(joinedRow);",
- " }",
- " }",
- " }",
- " return res;",
- " }",
- "",
- " @Override",
- " public void setSource(ChannelContext ctx, Object source) {",
- " this.source = source;",
- " }",
- "",
- " @Override",
- " public void flush(ChannelContext ctx) {",
- " if (source == left) {",
- " leftDone = true;",
- " } else if (source == right) {",
- " rightDone = true;",
- " }",
- " if (leftDone && rightDone) {",
- " if (leftRows.size() <= rightRows.size()) {",
- " for(Values res: join(getJoinTable(leftRows, ordinals[0]), rightRows, ordinals[1], false)) {",
- " ctx.emit(res);",
- " }",
- " } else {",
- " for(Values res: join(getJoinTable(rightRows, ordinals[1]), leftRows, ordinals[0], true)) {",
- " ctx.emit(res);",
- " }",
- " }",
- " leftDone = rightDone = false;",
- " leftRows.clear();",
- " rightRows.clear();",
- " super.flush(ctx);",
- " }",
- " }",
- "",
- " @Override",
- " public void dataReceived(ChannelContext ctx, Values _data) {",
- ""
- );
-
- private static final String STAGE_PASSTHROUGH = NEW_LINE_JOINER.join(
- " private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;",
- "");
-
- private static final String STAGE_ENUMERABLE_TABLE_SCAN = NEW_LINE_JOINER.join(
- " private static final ChannelHandler %1$s = new AbstractChannelHandler() {",
- " @Override",
- " public void flush(ChannelContext ctx) {",
- " ctx.setSource(this);",
- " super.flush(ctx);",
- " }",
- "",
- " @Override",
- " public void dataReceived(ChannelContext ctx, Values _data) {",
- " ctx.setSource(this);",
- " ctx.emit(_data);",
- " }",
- " };",
- "");
-
- private int nameCount;
- private Map<AggregateCall, String> aggregateCallVarNames = new HashMap<>();
-
- RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
- this.pw = pw;
- this.typeFactory = typeFactory;
- this.rexCompiler = new RexNodeToJavaCodeCompiler(new RexBuilder(typeFactory));
- }
-
- @Override
- public Void visitDelta(Delta delta, List<Void> inputStreams) throws Exception {
- pw.print(String.format(STAGE_PASSTHROUGH, getStageName(delta)));
- return null;
- }
-
- @Override
- public Void visitFilter(Filter filter, List<Void> inputStreams) throws Exception {
- beginStage(filter);
-
- List<RexNode> childExps = filter.getChildExps();
- RelDataType inputRowType = filter.getInput(0).getRowType();
-
- pw.print("Context context = new StormContext(Processor.dataContext);\n");
- pw.print("context.values = _data.toArray();\n");
- pw.print("Object[] outputValues = new Object[1];\n");
-
- pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
-
- String r = "((Boolean) outputValues[0])";
- if (filter.getCondition().getType().isNullable()) {
- pw.print(String.format(" if (%s != null && %s) { ctx.emit(_data); }\n", r, r));
- } else {
- pw.print(String.format(" if (%s) { ctx.emit(_data); }\n", r, r));
- }
- endStage();
- return null;
- }
-
- @Override
- public Void visitProject(Project project, List<Void> inputStreams) throws Exception {
- beginStage(project);
-
- List<RexNode> childExps = project.getChildExps();
- RelDataType inputRowType = project.getInput(0).getRowType();
- int outputCount = project.getRowType().getFieldCount();
-
- pw.print("Context context = new StormContext(Processor.dataContext);\n");
- pw.print("context.values = _data.toArray();\n");
- pw.print(String.format("Object[] outputValues = new Object[%d];\n", outputCount));
-
- pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
-
- pw.print(" ctx.emit(new Values(outputValues));\n");
- endStage();
- return null;
- }
-
- @Override
- public Void defaultValue(RelNode n, List<Void> inputStreams) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Void visitTableScan(TableScan scan, List<Void> inputStreams) throws Exception {
- pw.print(String.format(STAGE_ENUMERABLE_TABLE_SCAN, getStageName(scan)));
- return null;
- }
-
- @Override
- public Void visitAggregate(Aggregate aggregate, List<Void> inputStreams) throws Exception {
- beginAggregateStage(aggregate);
- pw.println(" if (_data != null) {");
- pw.println(" List<Object> curGroupValues = getGroupValues(_data);");
- pw.println(" if (!state.containsKey(curGroupValues)) {");
- pw.println(" state.put(curGroupValues, new HashMap<String, Object>());");
- pw.println(" }");
- pw.println(" Map<String, Object> accumulators = state.get(curGroupValues);");
- for (AggregateCall call : aggregate.getAggCallList()) {
- aggregate(call);
- }
- pw.println(" }");
- endStage();
- return null;
- }
-
- @Override
- public Void visitJoin(Join join, List<Void> inputStreams) {
- beginJoinStage(join);
- pw.println(" if (source == left) {");
- pw.println(" leftRows.add(_data);");
- pw.println(" } else if (source == right) {");
- pw.println(" rightRows.add(_data);");
- pw.println(" }");
- endStage();
- return null;
- }
-
- private String groupValueEmitStr(String var, int n) {
- int count = 0;
- StringBuilder sb = new StringBuilder();
- for (int i = 0; i < n; i++) {
- if (++count > 1) {
- sb.append(", ");
- }
- sb.append(var).append(".").append("get(").append(i).append(")");
- }
- return sb.toString();
- }
-
- private String emitAggregateStmts(Aggregate aggregate) {
- List<String> res = new ArrayList<>();
- StringWriter sw = new StringWriter();
- for (AggregateCall call : aggregate.getAggCallList()) {
- res.add(aggregateResult(call, new PrintWriter(sw)));
- }
- return NEW_LINE_JOINER.join(sw.toString(),
- String.format(" ctx.emit(new Values(%s, %s));",
- groupValueEmitStr("groupValues", aggregate.getGroupSet().cardinality()),
- Joiner.on(", ").join(res)));
- }
-
- private String aggregateResult(AggregateCall call, PrintWriter pw) {
- SqlAggFunction aggFunction = call.getAggregation();
- String aggregationName = call.getAggregation().getName();
- Type ty = typeFactory.getJavaClass(call.getType());
- String result;
- if (aggFunction instanceof SqlUserDefinedAggFunction) {
- AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
- result = doAggregateResult((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, pw);
- } else {
- List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
- if (typeClasses == null) {
- throw new UnsupportedOperationException(aggregationName + " Not implemented");
- }
- result = doAggregateResult(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
- reserveAggVarName(call), ty, pw);
- }
- return result;
- }
-
- private String doAggregateResult(AggregateFunctionImpl aggFn, String varName, Type ty, PrintWriter pw) {
- String resultName = varName + "_result";
- Class<?> accumulatorType = aggFn.accumulatorType;
- Class<?> resultType = aggFn.resultType;
- List<String> args = new ArrayList<>();
- if (!aggFn.isStatic) {
- String aggObjName = String.format("%s_obj", varName);
- String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
- pw.println(" @SuppressWarnings(\"unchecked\")");
- pw.println(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
- aggObjName));
- args.add(aggObjName);
- }
- args.add(String.format("(%s)accumulators.get(\"%s\")", accumulatorType.getCanonicalName(), varName));
- pw.println(String.format(" final %s %s = %s;", resultType.getCanonicalName(),
- resultName, printMethodCall(aggFn.resultMethod, args)));
-
- return resultName;
- }
-
- private void aggregate(AggregateCall call) {
- SqlAggFunction aggFunction = call.getAggregation();
- String aggregationName = call.getAggregation().getName();
- Type ty = typeFactory.getJavaClass(call.getType());
- if (call.getArgList().size() != 1) {
- if (aggregationName.equals("COUNT")) {
- if (call.getArgList().size() != 0) {
- throw new UnsupportedOperationException("Count with nullable fields");
- }
- }
- }
- if (aggFunction instanceof SqlUserDefinedAggFunction) {
- AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
- doAggregate((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, call.getArgList());
- } else {
- List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
- if (typeClasses == null) {
- throw new UnsupportedOperationException(aggregationName + " Not implemented");
- }
- doAggregate(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
- reserveAggVarName(call), ty, call.getArgList());
- }
- }
-
- private Class<?> findMatchingClass(String aggregationName, List<BuiltinAggregateFunctions.TypeClass> typeClasses, Type ty) {
- for (BuiltinAggregateFunctions.TypeClass typeClass : typeClasses) {
- if (typeClass.ty.equals(BuiltinAggregateFunctions.TypeClass.GenericType.class) || typeClass.ty.equals(ty)) {
- return typeClass.clazz;
- }
- }
- throw new UnsupportedOperationException(aggregationName + " Not implemeted for type '" + ty + "'");
- }
-
- private void doAggregate(AggregateFunctionImpl aggFn, String varName, Type ty, List<Integer> argList) {
- List<String> args = new ArrayList<>();
- Class<?> accumulatorType = aggFn.accumulatorType;
- if (!aggFn.isStatic) {
- String aggObjName = String.format("%s_obj", varName);
- String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
- pw.println(String.format(" if (!accumulators.containsKey(\"%s\")) { ", aggObjName));
- pw.println(String.format(" accumulators.put(\"%s\", new %s());", aggObjName, aggObjClassName));
- pw.println(" }");
- pw.println(" @SuppressWarnings(\"unchecked\")");
- pw.println(String.format(" final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
- aggObjName));
- args.add(aggObjName);
- }
- args.add(String.format("%1$s == null ? %2$s : (%3$s) %1$s",
- "accumulators.get(\"" + varName + "\")",
- printMethodCall(aggFn.initMethod, args),
- accumulatorType.getCanonicalName()));
- if (argList.isEmpty()) {
- args.add("EMPTY_VALUES");
- } else {
- for (int i = 0; i < aggFn.valueTypes.size(); i++) {
- args.add(String.format("(%s) %s", aggFn.valueTypes.get(i).getCanonicalName(), "_data.get(" + argList.get(i) + ")"));
- }
- }
- pw.print(String.format(" accumulators.put(\"%s\", %s);\n",
- varName,
- printMethodCall(aggFn.addMethod, args)));
- }
-
- private String reserveAggVarName(AggregateCall call) {
- String varName;
- if ((varName = aggregateCallVarNames.get(call)) == null) {
- varName = call.getAggregation().getName() + ++nameCount;
- aggregateCallVarNames.put(call, varName);
- }
- return varName;
- }
-
- private void beginStage(RelNode n) {
- pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
- }
-
- private void beginAggregateStage(Aggregate n) {
- pw.print(String.format(AGGREGATE_STAGE_PROLOGUE, getStageName(n), getGroupByIndices(n), emitAggregateStmts(n)));
- }
-
- private void beginJoinStage(Join join) {
- int[] ordinals = new int[2];
- if (!RelOptUtil.analyzeSimpleEquiJoin((LogicalJoin) join, ordinals)) {
- throw new UnsupportedOperationException("Only simple equi joins are supported");
- }
-
- pw.print(String.format(JOIN_STAGE_PROLOGUE, getStageName(join),
- getStageName(join.getLeft()),
- getStageName(join.getRight()),
- ordinals[0],
- ordinals[1]));
- }
-
- private void endStage() {
- pw.print(" }\n };\n");
- }
-
- static String getStageName(RelNode n) {
- return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
- }
-
- private String getGroupByIndices(Aggregate n) {
- StringBuilder res = new StringBuilder();
- int count = 0;
- for (int i : n.getGroupSet()) {
- if (++count > 1) {
- res.append(", ");
- }
- res.append(i);
- }
- return res.toString();
- }
-
- public static String printMethodCall(Method method, List<String> args) {
- return printMethodCall(method.getDeclaringClass(), method.getName(),
- Modifier.isStatic(method.getModifiers()), args);
- }
-
- private static String printMethodCall(Class<?> clazz, String method, boolean isStatic, List<String> args) {
- if (isStatic) {
- return String.format("%s.%s(%s)", clazz.getCanonicalName(), method, Joiner.on(',').join(args));
- } else {
- return String.format("%s.%s(%s)", args.get(0), method,
- Joiner.on(',').join(args.subList(1, args.size())));
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
deleted file mode 100644
index 0b7c053..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Copyright (C) 2010 Google, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.javac;
-
-
-import javax.tools.DiagnosticListener;
-import javax.tools.FileObject;
-import javax.tools.ForwardingJavaFileManager;
-import javax.tools.JavaCompiler;
-import javax.tools.JavaFileManager;
-import javax.tools.JavaFileObject;
-import javax.tools.SimpleJavaFileObject;
-import javax.tools.ToolProvider;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Collections.singleton;
-
-/**
- * This is a Java ClassLoader that will attempt to load a class from a string of source code.
- *
- * <h3>Example</h3>
- *
- * <pre>
- * String className = "com.foo.MyClass";
- * String classSource =
- * "package com.foo;\n" +
- * "public class MyClass implements Runnable {\n" +
- * " @Override public void run() {\n" +
- * " log(\"Hello world\");\n" +
- * " }\n" +
- * "}";
- *
- * // Load class from source.
- * ClassLoader classLoader = new CompilingClassLoader(
- * parentClassLoader, className, classSource);
- * Class myClass = classLoader.loadClass(className);
- *
- * // Use it.
- * Runnable instance = (Runnable)myClass.newInstance();
- * instance.run();
- * </pre>
- *
- * Only one chunk of source can be compiled per instance of CompilingClassLoader. If you need to
- * compile more, create multiple CompilingClassLoader instances.
- *
- * Uses Java 1.6's in built compiler API.
- *
- * If the class cannot be compiled, loadClass() will throw a ClassNotFoundException and log the
- * compile errors to System.err. If you don't want the messages logged, or want to explicitly handle
- * the messages you can provide your own {@link javax.tools.DiagnosticListener} through
- * {#setDiagnosticListener()}.
- *
- * @see java.lang.ClassLoader
- * @see javax.tools.JavaCompiler
- */
-public class CompilingClassLoader extends ClassLoader {
-
- /**
- * Thrown when code cannot be compiled.
- */
- public static class CompilerException extends Exception {
- private static final long serialVersionUID = -2936958840023603270L;
-
- public CompilerException(String message) {
- super(message);
- }
- }
-
- private final Map<String, ByteArrayOutputStream> byteCodeForClasses = new HashMap<>();
-
- private static final URI EMPTY_URI;
-
- static {
- try {
- // Needed to keep SimpleFileObject constructor happy.
- EMPTY_URI = new URI("");
- } catch (URISyntaxException e) {
- throw new Error(e);
- }
- }
-
- /**
- * @param parent Parent classloader to resolve dependencies from.
- * @param className Name of class to compile. eg. "com.foo.MyClass".
- * @param sourceCode Java source for class. e.g. "package com.foo; class MyClass { ... }".
- * @param diagnosticListener Notified of compiler errors (may be null).
- */
- public CompilingClassLoader(
- ClassLoader parent,
- String className,
- String sourceCode,
- DiagnosticListener<JavaFileObject> diagnosticListener)
- throws CompilerException {
- super(parent);
- if (!compileSourceCodeToByteCode(className, sourceCode, diagnosticListener)) {
- throw new CompilerException("Could not compile " + className);
- }
- }
-
- public Map<String, ByteArrayOutputStream> getClasses() {
- return byteCodeForClasses;
- }
-
- /**
- * Override ClassLoader's class resolving method. Don't call this directly, instead use
- * {@link ClassLoader#loadClass(String)}.
- */
- @Override
- public Class<?> findClass(String name) throws ClassNotFoundException {
- ByteArrayOutputStream byteCode = byteCodeForClasses.get(name);
- if (byteCode == null) {
- throw new ClassNotFoundException(name);
- }
- return defineClass(name, byteCode.toByteArray(), 0, byteCode.size());
- }
-
- /**
- * @return Whether compilation was successful.
- */
- private boolean compileSourceCodeToByteCode(
- String className, String sourceCode, DiagnosticListener<JavaFileObject> diagnosticListener) {
- JavaCompiler javaCompiler = ToolProvider.getSystemJavaCompiler();
-
- // Set up the in-memory filesystem.
- InMemoryFileManager fileManager =
- new InMemoryFileManager(javaCompiler.getStandardFileManager(null, null, null));
- JavaFileObject javaFile = new InMemoryJavaFile(className, sourceCode);
-
- // Javac option: remove these when the javac zip impl is fixed
- // (http://b/issue?id=1822932)
- System.setProperty("useJavaUtilZip", "true"); // setting value to any non-null string
- List<String> options = new LinkedList<>();
- // this is ignored by javac currently but useJavaUtilZip should be
- // a valid javac XD option, which is another bug
- options.add("-XDuseJavaUtilZip");
-
- // Now compile!
- JavaCompiler.CompilationTask compilationTask =
- javaCompiler.getTask(
- null, // Null: log any unhandled errors to stderr.
- fileManager,
- diagnosticListener,
- options,
- null,
- singleton(javaFile));
- return compilationTask.call();
- }
-
- /**
- * Provides an in-memory representation of JavaFileManager abstraction, so we do not need to write
- * any files to disk.
- *
- * When files are written to, rather than putting the bytes on disk, they are appended to buffers
- * in byteCodeForClasses.
- *
- * @see javax.tools.JavaFileManager
- */
- private class InMemoryFileManager extends ForwardingJavaFileManager<JavaFileManager> {
- public InMemoryFileManager(JavaFileManager fileManager) {
- super(fileManager);
- }
-
- @Override
- public JavaFileObject getJavaFileForOutput(
- Location location, final String className, JavaFileObject.Kind kind, FileObject sibling)
- throws IOException {
- return new SimpleJavaFileObject(EMPTY_URI, kind) {
- @Override
- public OutputStream openOutputStream() throws IOException {
- ByteArrayOutputStream outputStream = byteCodeForClasses.get(className);
- if (outputStream != null) {
- throw new IllegalStateException("Cannot write more than once");
- }
- // Reasonable size for a simple .class.
- outputStream = new ByteArrayOutputStream(256);
- byteCodeForClasses.put(className, outputStream);
- return outputStream;
- }
- };
- }
- }
-
- private static class InMemoryJavaFile extends SimpleJavaFileObject {
- private final String sourceCode;
-
- public InMemoryJavaFile(String className, String sourceCode) {
- super(makeUri(className), Kind.SOURCE);
- this.sourceCode = sourceCode;
- }
-
- private static URI makeUri(String className) {
- try {
- return new URI(className.replaceAll("\\.", "/") + Kind.SOURCE.extension);
- } catch (URISyntaxException e) {
- throw new RuntimeException(e); // Not sure what could cause this.
- }
- }
-
- @Override
- public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
- return sourceCode;
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
deleted file mode 100644
index c67d8e7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlMonotonicity;
-
-public class ColumnConstraint extends SqlLiteral {
- private ColumnConstraint(
- Object value, SqlTypeName typeName, SqlParserPos pos) {
- super(value, typeName, pos);
- }
-
- public static class PrimaryKey extends ColumnConstraint {
- private final SqlMonotonicity monotonicity;
- public PrimaryKey(SqlMonotonicity monotonicity, SqlParserPos pos) {
- super(SqlDDLKeywords.PRIMARY, SqlTypeName.SYMBOL, pos);
- this.monotonicity = monotonicity;
- }
- public SqlMonotonicity monotonicity() {
- return monotonicity;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
deleted file mode 100644
index 3520b86..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.parser.SqlParserPos;
-
-import java.util.Arrays;
-
-public class ColumnDefinition extends SqlNodeList {
- public ColumnDefinition(
- SqlIdentifier name, SqlDataTypeSpec type, ColumnConstraint constraint, SqlParserPos pos) {
- super(Arrays.asList(name, type, constraint), pos);
- }
-
- public String name() {
- return get(0).toString();
- }
-
- public SqlDataTypeSpec type() {
- return (SqlDataTypeSpec) get(1);
- }
-
- public ColumnConstraint constraint() {
- return (ColumnConstraint) get(2);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
deleted file mode 100644
index a53802c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.NlsString;
-
-import java.util.List;
-
-public class SqlCreateFunction extends SqlCall {
- public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
- "CREATE_FUNCTION", SqlKind.OTHER) {
- @Override
- public SqlCall createCall(
- SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
- assert functionQualifier == null;
- return new SqlCreateFunction(pos, (SqlIdentifier) o[0], o[1], o[2]);
- }
-
- @Override
- public void unparse(
- SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
- SqlCreateFunction t = (SqlCreateFunction) call;
- UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
- u.keyword("CREATE", "FUNCTION").node(t.functionName).keyword("AS").node(t.className);
- if (t.jarName != null) {
- u.keyword("USING", "JAR").node(t.jarName);
- }
- }
- };
-
- private final SqlIdentifier functionName;
- private final SqlNode className;
- private final SqlNode jarName;
-
- public SqlCreateFunction(SqlParserPos pos, SqlIdentifier functionName, SqlNode className, SqlNode jarName) {
- super(pos);
- this.functionName = functionName;
- this.className = className;
- this.jarName = jarName;
- }
-
- @Override
- public SqlOperator getOperator() {
- return OPERATOR;
- }
-
- @Override
- public List<SqlNode> getOperandList() {
- return ImmutableNullableList.of(functionName, className);
- }
-
-
- @Override
- public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
- getOperator().unparse(writer, this, leftPrec, rightPrec);
- }
-
- public String functionName() {
- return functionName.toString();
- }
-
- public String className() {
- return ((NlsString)SqlLiteral.value(className)).getValue();
- }
-
- public String jarName() {
- return jarName == null ? null : ((NlsString)SqlLiteral.value(jarName)).getValue();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
deleted file mode 100644
index 670eedb..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-
-public class SqlCreateTable extends SqlCall {
- private static final int DEFAULT_PARALLELISM = 1;
-
- public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
- "CREATE_TABLE", SqlKind.OTHER) {
- @Override
- public SqlCall createCall(
- SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
- assert functionQualifier == null;
- return new SqlCreateTable(pos, (SqlIdentifier) o[0], (SqlNodeList) o[1],
- o[2], o[3], o[4], o[5], o[6], o[7]);
- }
-
- @Override
- public void unparse(
- SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
- SqlCreateTable t = (SqlCreateTable) call;
- UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
- u.keyword("CREATE", "EXTERNAL", "TABLE").node(t.tblName).nodeList(
- t.fieldList);
- if (t.inputFormatClass != null && t.outputFormatClass != null) {
- u.keyword("STORED", "AS", "INPUTFORMAT").node(
- t.inputFormatClass).keyword("OUTPUTFORMAT").node(
- t.outputFormatClass);
- }
- u.keyword("LOCATION").node(t.location);
- if (t.parallelism != null) {
- u.keyword("PARALLELISM").node(t.parallelism);
- }
- if (t.properties != null) {
- u.keyword("TBLPROPERTIES").node(t.properties);
- }
- if (t.query != null) {
- u.keyword("AS").node(t.query);
- }
- }
- };
-
- private final SqlIdentifier tblName;
- private final SqlNodeList fieldList;
- private final SqlNode inputFormatClass;
- private final SqlNode outputFormatClass;
- private final SqlNode location;
- private final SqlNode parallelism;
- private final SqlNode properties;
- private final SqlNode query;
-
- public SqlCreateTable(
- SqlParserPos pos, SqlIdentifier tblName, SqlNodeList fieldList,
- SqlNode inputFormatClass, SqlNode outputFormatClass, SqlNode location,
- SqlNode parallelism, SqlNode properties, SqlNode query) {
- super(pos);
- this.tblName = tblName;
- this.fieldList = fieldList;
- this.inputFormatClass = inputFormatClass;
- this.outputFormatClass = outputFormatClass;
- this.location = location;
- this.parallelism = parallelism;
- this.properties = properties;
- this.query = query;
- }
-
- @Override
- public SqlOperator getOperator() {
- return OPERATOR;
- }
-
- @Override
- public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
- getOperator().unparse(writer, this, leftPrec, rightPrec);
- }
-
- @Override
- public List<SqlNode> getOperandList() {
- return ImmutableNullableList.of(tblName, fieldList, inputFormatClass,
- outputFormatClass, location, properties,
- query);
- }
-
- public String tableName() {
- return tblName.toString();
- }
-
- public URI location() {
- return URI.create(getString(location));
- }
-
- public Integer parallelism() {
- String parallelismStr = getString(parallelism);
- if (parallelismStr != null) {
- return Integer.parseInt(parallelismStr);
- } else {
- return DEFAULT_PARALLELISM;
- }
- }
-
- public String inputFormatClass() {
- return getString(inputFormatClass);
- }
-
- public String outputFormatClass() {
- return getString(outputFormatClass);
- }
-
- public Properties properties() {
- Properties props = new Properties();
- if (properties != null) {
- try {
- ObjectMapper mapper = new ObjectMapper();
- HashMap<String, Object> map = mapper.readValue(getString(properties), HashMap.class);
- props.putAll(map);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- return props;
- }
-
- private String getString(SqlNode n) {
- return n == null ? null : SqlLiteral.stringValue(n);
- }
-
- @SuppressWarnings("unchecked")
- public List<ColumnDefinition> fieldList() {
- return (List<ColumnDefinition>)((List<? extends SqlNode>)fieldList.getList());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
deleted file mode 100644
index 3112e53..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlLiteral;
-
-/**
- * Define the keywords that can occur in a CREATE TABLE statement
- */
-public enum SqlDDLKeywords implements SqlLiteral.SqlSymbol {
- PRIMARY
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
deleted file mode 100644
index 8444e1e..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.calcite.config.Lex;
-import org.apache.storm.sql.parser.impl.StormParserImpl;
-
-import java.io.StringReader;
-
-public class StormParser {
- public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
- private final StormParserImpl impl;
-
- public StormParser(String s) {
- this.impl = new StormParserImpl(new StringReader(s));
- this.impl.setTabSize(1);
- this.impl.setQuotedCasing(Lex.ORACLE.quotedCasing);
- this.impl.setUnquotedCasing(Lex.ORACLE.unquotedCasing);
- this.impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH);
- /*
- * By default parser uses [ ] for quoting identifiers. Switching to DQID (double quoted identifiers)
- * is needed for array and map access (m['x'] = 1 or arr[2] = 10 etc) to work.
- */
- this.impl.switchTo("DQID");
- }
-
- @VisibleForTesting
- public StormParserImpl impl() {
- return impl;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
deleted file mode 100644
index 834fe7c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlWriter;
-
-class UnparseUtil {
- private final SqlWriter writer;
- private final int leftPrec;
- private final int rightPrec;
-
- UnparseUtil(SqlWriter writer, int leftPrec, int rightPrec) {
- this.writer = writer;
- this.leftPrec = leftPrec;
- this.rightPrec = rightPrec;
- }
-
- UnparseUtil keyword(String... keywords) {
- for (String k : keywords) {
- writer.keyword(k);
- }
- return this;
- }
-
- UnparseUtil node(SqlNode n) {
- n.unparse(writer, leftPrec, rightPrec);
- return this;
- }
-
- UnparseUtil nodeList(SqlNodeList l) {
- writer.keyword("(");
- if (l.size() > 0) {
- l.get(0).unparse(writer, leftPrec, rightPrec);
- for (int i = 1; i < l.size(); ++i) {
- writer.keyword(",");
- l.get(i).unparse(writer, leftPrec, rightPrec);
- }
- }
- writer.keyword(")");
- return this;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
deleted file mode 100644
index 437877c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner;
-
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-
-public class StormRelDataTypeSystem extends RelDataTypeSystemImpl {
- public static final RelDataTypeSystem STORM_REL_DATATYPE_SYSTEM = new StormRelDataTypeSystem();
-
- @Override
- public int getMaxNumericScale() {
- return 38;
- }
-
- @Override
- public int getMaxNumericPrecision() {
- return 38;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
deleted file mode 100644
index 40bbacd..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner;
-
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class StormRelUtils {
- private static final Logger LOG = LoggerFactory.getLogger(StormRelUtils.class);
-
- private static final AtomicInteger sequence = new AtomicInteger(0);
- private static final AtomicInteger classSequence = new AtomicInteger(0);
-
- public static String getStageName(TridentRel relNode) {
- return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" + sequence.getAndIncrement();
- }
-
- public static String getClassName(TridentRel relNode) {
- return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" +
- classSequence.getAndIncrement();
- }
-
- public static TridentRel getStormRelInput(RelNode input) {
- if (input instanceof RelSubset) {
- // go with known best input
- input = ((RelSubset) input).getBest();
- }
- return (TridentRel) input;
- }
-
- public static String explain(final RelNode rel) {
- return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
- }
-
- public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
- String explain = "";
- try {
- explain = RelOptUtil.toString(rel);
- } catch (StackOverflowError e) {
- LOG.error("StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list.");
- LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
- LOG.error("Forcing plan to empty string and continue... SQL Runner may not working properly after.");
- }
- return explain;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
deleted file mode 100644
index 258fe72..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner;
-
-import org.apache.calcite.sql.util.SqlShuttle;
-
-public class UnsupportedOperatorsVisitor extends SqlShuttle {
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
deleted file mode 100644
index 1ea7912..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormCalcRelBase extends Calc implements StormRelNode {
- protected StormCalcRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
- super(cluster, traits, child, program);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
deleted file mode 100644
index 4e28460..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormFilterRelBase extends Filter implements StormRelNode {
- protected StormFilterRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
- super(cluster, traits, child, condition);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
deleted file mode 100644
index 1458ce7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.Set;
-
-public abstract class StormJoinRelBase extends Join implements StormRelNode {
- protected StormJoinRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
- super(cluster, traitSet, left, right, condition, variablesSet, joinType);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
deleted file mode 100644
index f4f23dc..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.List;
-
-public abstract class StormProjectRelBase extends Project implements StormRelNode {
- protected StormProjectRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
- super(cluster, traits, input, projects, rowType);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
deleted file mode 100644
index 9327868..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.rel.RelNode;
-
-public interface StormRelNode extends RelNode {
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
deleted file mode 100644
index df47f3f..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.List;
-
-public abstract class StormStreamInsertRelBase extends TableModify implements StormRelNode {
- protected StormStreamInsertRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
- super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
deleted file mode 100644
index 36c62b2..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormStreamScanRelBase extends TableScan implements StormRelNode {
-
- // FIXME: define Table class and table.unwrap() to get it
-
- protected StormStreamScanRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
- super(cluster, traitSet, table);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
deleted file mode 100644
index f98fb02..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- *
- * * Licensed to the Apache Software Foundation (ASF) under one
- * * or more contributor license agreements. See the NOTICE file
- * * distributed with this work for additional information
- * * regarding copyright ownership. The ASF licenses this file
- * * to you under the Apache License, Version 2.0 (the
- * * "License"); you may not use this file except in compliance
- * * with the License. You may obtain a copy of the License at
- * *
- * * http://www.apache.org/licenses/LICENSE-2.0
- * *
- * * Unless required by applicable law or agreed to in writing, software
- * * distributed under the License is distributed on an "AS IS" BASIS,
- * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * * See the License for the specific language governing permissions and
- * * limitations under the License.
- *
- */
-package org.apache.storm.sql.planner.trident;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.planner.StormRelDataTypeSystem;
-import org.apache.storm.sql.planner.UnsupportedOperatorsVisitor;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.AbstractTridentProcessor;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class QueryPlanner {
-
- public static final int STORM_REL_CONVERSION_RULES = 1;
-
- private final Planner planner;
-
- private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
- RelDataTypeSystem.DEFAULT);
-
- public QueryPlanner(SchemaPlus schema) {
- final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
-
- traitDefs.add(ConventionTraitDef.INSTANCE);
- traitDefs.add(RelCollationTraitDef.INSTANCE);
-
- List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
- sqlOperatorTables.add(SqlStdOperatorTable.instance());
- sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
- false,
- Collections.<String>emptyList(), typeFactory));
-
- FrameworkConfig config = Frameworks.newConfigBuilder()
- .defaultSchema(schema)
- .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
- .traitDefs(traitDefs)
- .context(Contexts.EMPTY_CONTEXT)
- .ruleSets(TridentStormRuleSets.getRuleSets())
- .costFactory(null)
- .typeSystem(StormRelDataTypeSystem.STORM_REL_DATATYPE_SYSTEM)
- .build();
- this.planner = Frameworks.getPlanner(config);
- }
-
- public AbstractTridentProcessor compile(Map<String, ISqlTridentDataSource> sources, String query) throws Exception {
- TridentRel relNode = getPlan(query);
-
- TridentPlanCreator tridentPlanCreator = new TridentPlanCreator(sources, new RexBuilder(typeFactory));
- relNode.tridentPlan(tridentPlanCreator);
-
- final TridentTopology topology = tridentPlanCreator.getTopology();
- final IAggregatableStream lastStream = tridentPlanCreator.pop();
- final DataContext dc = tridentPlanCreator.getDataContext();
- final List<CompilingClassLoader> cls = tridentPlanCreator.getClassLoaders();
-
- return new AbstractTridentProcessor() {
- @Override
- public TridentTopology build() {
- return topology;
- }
-
- @Override
- public Stream outputStream() {
- return lastStream.toStream();
- }
-
- @Override
- public DataContext getDataContext() {
- return dc;
- }
-
- @Override
- public List<CompilingClassLoader> getClassLoaders() {
- return cls;
- }
- };
- }
-
- public TridentRel getPlan(String query) throws ValidationException, RelConversionException, SqlParseException {
- return (TridentRel) validateAndConvert(planner.parse(query));
- }
-
- private RelNode validateAndConvert(SqlNode sqlNode) throws ValidationException, RelConversionException {
- SqlNode validated = validateNode(sqlNode);
- RelNode relNode = convertToRelNode(validated);
- return convertToStormRel(relNode);
- }
-
- private RelNode convertToStormRel(RelNode relNode) throws RelConversionException {
- RelTraitSet traitSet = relNode.getTraitSet();
- traitSet = traitSet.simplify();
-
- // PlannerImpl.transform() optimizes RelNode with ruleset
- return planner.transform(STORM_REL_CONVERSION_RULES, traitSet.plus(TridentLogicalConvention.INSTANCE), relNode);
- }
-
- private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
- return planner.rel(sqlNode).rel;
- }
-
- private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
- SqlNode validatedSqlNode = planner.validate(sqlNode);
- validatedSqlNode.accept(new UnsupportedOperatorsVisitor());
- return validatedSqlNode;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
deleted file mode 100644
index aa30552..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.interpreter.Scalar;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.StormDataContext;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Deque;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-public class TridentPlanCreator {
- private final Map<String, ISqlTridentDataSource> sources;
- private final JavaTypeFactory typeFactory;
- private final RexNodeToJavaCodeCompiler rexCompiler;
- private final DataContext dataContext;
- private final TridentTopology topology;
-
- private final Deque<IAggregatableStream> streamStack = new ArrayDeque<>();
- private final List<CompilingClassLoader> classLoaders = new ArrayList<>();
-
- public TridentPlanCreator(Map<String, ISqlTridentDataSource> sources, RexBuilder rexBuilder) {
- this.sources = sources;
- this.rexCompiler = new RexNodeToJavaCodeCompiler(rexBuilder);
- this.typeFactory = (JavaTypeFactory) rexBuilder.getTypeFactory();
-
- this.topology = new TridentTopology();
- this.dataContext = new StormDataContext();
- }
-
- public void addStream(IAggregatableStream stream) throws Exception {
- push(stream);
- }
-
- public IAggregatableStream pop() {
- return streamStack.pop();
- }
-
- public Map<String, ISqlTridentDataSource> getSources() {
- return sources;
- }
-
- public DataContext getDataContext() {
- return dataContext;
- }
-
- public JavaTypeFactory getTypeFactory() {
- return typeFactory;
- }
-
- public TridentTopology getTopology() {
- return topology;
- }
-
- public ExecutableExpression createScalarInstance(List<RexNode> nodes, RelDataType inputRowType, String className)
- throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
- String expr = rexCompiler.compile(nodes, inputRowType, className);
- CompilingClassLoader classLoader = new CompilingClassLoader(
- getLastClassLoader(), className, expr, null);
- ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
- addClassLoader(classLoader);
- return new DebuggableExecutableExpression(instance, expr);
- }
-
- public ExecutableExpression createScalarInstance(RexProgram program, String className)
- throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
- String expr = rexCompiler.compile(program, className);
- CompilingClassLoader classLoader = new CompilingClassLoader(
- getLastClassLoader(), className, expr, null);
- ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
- addClassLoader(classLoader);
- return new DebuggableExecutableExpression(instance, expr);
- }
-
- private void push(IAggregatableStream stream) {
- streamStack.push(stream);
- }
-
- public void addClassLoader(CompilingClassLoader compilingClassLoader) {
- this.classLoaders.add(compilingClassLoader);
- }
-
- public ClassLoader getLastClassLoader() {
- if (this.classLoaders.size() > 0) {
- return this.classLoaders.get(this.classLoaders.size() - 1);
- } else {
- return this.getClass().getClassLoader();
- }
- }
-
- public List<CompilingClassLoader> getClassLoaders() {
- return classLoaders;
- }
-}
[20/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
deleted file mode 100644
index e79dfb7..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
+++ /dev/null
@@ -1,630 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.grouping.CustomStreamGrouping;
-import org.apache.storm.topology.*;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.flux.model.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.*;
-import java.util.*;
-
-public class FluxBuilder {
- private static Logger LOG = LoggerFactory.getLogger(FluxBuilder.class);
-
- /**
- * Given a topology definition, return a populated `org.apache.storm.Config` instance.
- *
- * @param topologyDef
- * @return
- */
- public static Config buildConfig(TopologyDef topologyDef) {
- // merge contents of `config` into topology config
- Config conf = new Config();
- conf.putAll(topologyDef.getConfig());
- return conf;
- }
-
- /**
- * Given a topology definition, return a Storm topology that can be run either locally or remotely.
- *
- * @param context
- * @return
- * @throws IllegalAccessException
- * @throws InstantiationException
- * @throws ClassNotFoundException
- * @throws NoSuchMethodException
- * @throws InvocationTargetException
- */
- public static StormTopology buildTopology(ExecutionContext context) throws IllegalAccessException,
- InstantiationException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-
- StormTopology topology = null;
- TopologyDef topologyDef = context.getTopologyDef();
-
- if(!topologyDef.validate()){
- throw new IllegalArgumentException("Invalid topology config. Spouts, bolts and streams cannot be " +
- "defined in the same configuration as a topologySource.");
- }
-
- // build components that may be referenced by spouts, bolts, etc.
- // the map will be a String --> Object where the object is a fully
- // constructed class instance
- buildComponents(context);
-
- if(topologyDef.isDslTopology()) {
- // This is a DSL (YAML, etc.) topology...
- LOG.info("Detected DSL topology...");
-
- TopologyBuilder builder = new TopologyBuilder();
-
- // create spouts
- buildSpouts(context, builder);
-
- // we need to be able to lookup bolts by id, then switch based
- // on whether they are IBasicBolt or IRichBolt instances
- buildBolts(context);
-
- // process stream definitions
- buildStreamDefinitions(context, builder);
-
- topology = builder.createTopology();
- } else {
- // user class supplied...
- // this also provides a bridge to Trident...
- LOG.info("A topology source has been specified...");
- ObjectDef def = topologyDef.getTopologySource();
- topology = buildExternalTopology(def, context);
- }
- return topology;
- }
-
- /**
- * Given a `java.lang.Object` instance and a method name, attempt to find a method that matches the input
- * parameter: `java.util.Map` or `org.apache.storm.Config`.
- *
- * @param topologySource object to inspect for the specified method
- * @param methodName name of the method to look for
- * @return
- * @throws NoSuchMethodException
- */
- private static Method findGetTopologyMethod(Object topologySource, String methodName) throws NoSuchMethodException {
- Class clazz = topologySource.getClass();
- Method[] methods = clazz.getMethods();
- ArrayList<Method> candidates = new ArrayList<Method>();
- for(Method method : methods){
- if(!method.getName().equals(methodName)){
- continue;
- }
- if(!method.getReturnType().equals(StormTopology.class)){
- continue;
- }
- Class[] paramTypes = method.getParameterTypes();
- if(paramTypes.length != 1){
- continue;
- }
- if(paramTypes[0].isAssignableFrom(Map.class) || paramTypes[0].isAssignableFrom(Config.class)){
- candidates.add(method);
- }
- }
-
- if(candidates.size() == 0){
- throw new IllegalArgumentException("Unable to find method '" + methodName + "' method in class: " + clazz.getName());
- } else if (candidates.size() > 1){
- LOG.warn("Found multiple candidate methods in class '" + clazz.getName() + "'. Using the first one found");
- }
-
- return candidates.get(0);
- }
-
- /**
- * @param context
- * @param builder
- */
- private static void buildStreamDefinitions(ExecutionContext context, TopologyBuilder builder)
- throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException,
- IllegalAccessException, NoSuchFieldException {
- TopologyDef topologyDef = context.getTopologyDef();
- // process stream definitions
- HashMap<String, BoltDeclarer> declarers = new HashMap<String, BoltDeclarer>();
- for (StreamDef stream : topologyDef.getStreams()) {
- Object boltObj = context.getBolt(stream.getTo());
- BoltDeclarer declarer = declarers.get(stream.getTo());
- if (boltObj instanceof IRichBolt) {
- if(declarer == null) {
- declarer = builder.setBolt(stream.getTo(),
- (IRichBolt) boltObj,
- topologyDef.parallelismForBolt(stream.getTo()));
- declarers.put(stream.getTo(), declarer);
- }
- } else if (boltObj instanceof IBasicBolt) {
- if(declarer == null) {
- declarer = builder.setBolt(
- stream.getTo(),
- (IBasicBolt) boltObj,
- topologyDef.parallelismForBolt(stream.getTo()));
- declarers.put(stream.getTo(), declarer);
- }
- } else if (boltObj instanceof IWindowedBolt) {
- if(declarer == null) {
- declarer = builder.setBolt(
- stream.getTo(),
- (IWindowedBolt) boltObj,
- topologyDef.parallelismForBolt(stream.getTo()));
- declarers.put(stream.getTo(), declarer);
- }
- } else if (boltObj instanceof IStatefulBolt) {
- if(declarer == null) {
- declarer = builder.setBolt(
- stream.getTo(),
- (IStatefulBolt) boltObj,
- topologyDef.parallelismForBolt(stream.getTo()));
- declarers.put(stream.getTo(), declarer);
- }
- } else {
- throw new IllegalArgumentException("Class does not appear to be a bolt: " +
- boltObj.getClass().getName());
- }
-
- GroupingDef grouping = stream.getGrouping();
- // if the streamId is defined, use it for the grouping, otherwise assume storm's default stream
- String streamId = (grouping.getStreamId() == null ? Utils.DEFAULT_STREAM_ID : grouping.getStreamId());
-
-
- switch (grouping.getType()) {
- case SHUFFLE:
- declarer.shuffleGrouping(stream.getFrom(), streamId);
- break;
- case FIELDS:
- //TODO check for null grouping args
- declarer.fieldsGrouping(stream.getFrom(), streamId, new Fields(grouping.getArgs()));
- break;
- case ALL:
- declarer.allGrouping(stream.getFrom(), streamId);
- break;
- case DIRECT:
- declarer.directGrouping(stream.getFrom(), streamId);
- break;
- case GLOBAL:
- declarer.globalGrouping(stream.getFrom(), streamId);
- break;
- case LOCAL_OR_SHUFFLE:
- declarer.localOrShuffleGrouping(stream.getFrom(), streamId);
- break;
- case NONE:
- declarer.noneGrouping(stream.getFrom(), streamId);
- break;
- case CUSTOM:
- declarer.customGrouping(stream.getFrom(), streamId,
- buildCustomStreamGrouping(stream.getGrouping().getCustomClass(), context));
- break;
- default:
- throw new UnsupportedOperationException("unsupported grouping type: " + grouping);
- }
- }
- }
-
- private static void applyProperties(ObjectDef bean, Object instance, ExecutionContext context) throws
- IllegalAccessException, InvocationTargetException, NoSuchFieldException {
- List<PropertyDef> props = bean.getProperties();
- Class clazz = instance.getClass();
- if (props != null) {
- for (PropertyDef prop : props) {
- Object value = prop.isReference() ? context.getComponent(prop.getRef()) : prop.getValue();
- Method setter = findSetter(clazz, prop.getName(), value);
- if (setter != null) {
- LOG.debug("found setter, attempting to invoke");
- // invoke setter
- setter.invoke(instance, new Object[]{value});
- } else {
- // look for a public instance variable
- LOG.debug("no setter found. Looking for a public instance variable...");
- Field field = findPublicField(clazz, prop.getName(), value);
- if (field != null) {
- field.set(instance, value);
- }
- }
- }
- }
- }
-
- private static Field findPublicField(Class clazz, String property, Object arg) throws NoSuchFieldException {
- Field field = clazz.getField(property);
- return field;
- }
-
- private static Method findSetter(Class clazz, String property, Object arg) {
- String setterName = toSetterName(property);
- Method retval = null;
- Method[] methods = clazz.getMethods();
- for (Method method : methods) {
- if (setterName.equals(method.getName())) {
- LOG.debug("Found setter method: " + method.getName());
- retval = method;
- }
- }
- return retval;
- }
-
- private static String toSetterName(String name) {
- return "set" + name.substring(0, 1).toUpperCase() + name.substring(1, name.length());
- }
-
- private static List<Object> resolveReferences(List<Object> args, ExecutionContext context) {
- LOG.debug("Checking arguments for references.");
- List<Object> cArgs = new ArrayList<Object>();
- // resolve references
- for (Object arg : args) {
- if (arg instanceof BeanReference) {
- cArgs.add(context.getComponent(((BeanReference) arg).getId()));
- } else if (arg instanceof BeanListReference) {
- List<Object> components = new ArrayList<>();
- BeanListReference ref = (BeanListReference) arg;
- for (String id : ref.getIds()) {
- components.add(context.getComponent(id));
- }
-
- LOG.debug("BeanListReference resolved as {}", components);
- cArgs.add(components);
- } else {
- cArgs.add(arg);
- }
- }
- return cArgs;
- }
-
- private static Object buildObject(ObjectDef def, ExecutionContext context) throws ClassNotFoundException,
- IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
- Class clazz = Class.forName(def.getClassName());
- Object obj = null;
- if (def.hasConstructorArgs()) {
- LOG.debug("Found constructor arguments in definition: " + def.getConstructorArgs().getClass().getName());
- List<Object> cArgs = def.getConstructorArgs();
- if(def.hasReferences()){
- cArgs = resolveReferences(cArgs, context);
- }
- Constructor con = findCompatibleConstructor(cArgs, clazz);
- if (con != null) {
- LOG.debug("Found something seemingly compatible, attempting invocation...");
- obj = con.newInstance(getArgsWithListCoercian(cArgs, con.getParameterTypes()));
- } else {
- String msg = String.format("Couldn't find a suitable constructor for class '%s' with arguments '%s'.",
- clazz.getName(),
- cArgs);
- throw new IllegalArgumentException(msg);
- }
- } else {
- obj = clazz.newInstance();
- }
- applyProperties(def, obj, context);
- invokeConfigMethods(def, obj, context);
- return obj;
- }
-
- private static StormTopology buildExternalTopology(ObjectDef def, ExecutionContext context)
- throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException,
- InvocationTargetException, NoSuchFieldException {
-
- Object topologySource = buildObject(def, context);
-
- String methodName = context.getTopologyDef().getTopologySource().getMethodName();
- Method getTopology = findGetTopologyMethod(topologySource, methodName);
- if(getTopology.getParameterTypes()[0].equals(Config.class)){
- Config config = new Config();
- config.putAll(context.getTopologyDef().getConfig());
- return (StormTopology) getTopology.invoke(topologySource, config);
- } else {
- return (StormTopology) getTopology.invoke(topologySource, context.getTopologyDef().getConfig());
- }
- }
-
- private static CustomStreamGrouping buildCustomStreamGrouping(ObjectDef def, ExecutionContext context)
- throws ClassNotFoundException,
- IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
- Object grouping = buildObject(def, context);
- return (CustomStreamGrouping)grouping;
- }
-
- /**
- * Given a topology definition, resolve and instantiate all components found and return a map
- * keyed by the component id.
- */
- private static void buildComponents(ExecutionContext context) throws ClassNotFoundException, NoSuchMethodException,
- IllegalAccessException, InvocationTargetException, InstantiationException, NoSuchFieldException {
- Collection<BeanDef> cDefs = context.getTopologyDef().getComponents();
- if (cDefs != null) {
- for (BeanDef bean : cDefs) {
- Object obj = buildObject(bean, context);
- context.addComponent(bean.getId(), obj);
- }
- }
- }
-
-
- private static void buildSpouts(ExecutionContext context, TopologyBuilder builder) throws ClassNotFoundException,
- NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, NoSuchFieldException {
- for (SpoutDef sd : context.getTopologyDef().getSpouts()) {
- IRichSpout spout = buildSpout(sd, context);
- builder.setSpout(sd.getId(), spout, sd.getParallelism());
- context.addSpout(sd.getId(), spout);
- }
- }
-
- /**
- * Given a spout definition, return a Storm spout implementation by attempting to find a matching constructor
- * in the given spout class. Perform list to array conversion as necessary.
- */
- private static IRichSpout buildSpout(SpoutDef def, ExecutionContext context) throws ClassNotFoundException,
- IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
- return (IRichSpout)buildObject(def, context);
- }
-
- /**
- * Given a list of bolt definitions, build a map of Storm bolts with the bolt definition id as the key.
- * Attempt to coerce the given constructor arguments to a matching bolt constructor as much as possible.
- */
- private static void buildBolts(ExecutionContext context) throws ClassNotFoundException, IllegalAccessException,
- InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
- for (BoltDef def : context.getTopologyDef().getBolts()) {
- Class clazz = Class.forName(def.getClassName());
- Object bolt = buildObject(def, context);
- context.addBolt(def.getId(), bolt);
- }
- }
-
- /**
- * Given a list of constructor arguments, and a target class, attempt to find a suitable constructor.
- *
- */
- private static Constructor findCompatibleConstructor(List<Object> args, Class target) throws NoSuchMethodException {
- Constructor retval = null;
- int eligibleCount = 0;
-
- LOG.debug("Target class: {}, constructor args: {}", target.getName(), args);
- Constructor[] cons = target.getDeclaredConstructors();
-
- for (Constructor con : cons) {
- Class[] paramClasses = con.getParameterTypes();
- if (paramClasses.length == args.size()) {
- LOG.debug("found constructor with same number of args..");
- boolean invokable = canInvokeWithArgs(args, con.getParameterTypes());
- if (invokable) {
- retval = con;
- eligibleCount++;
- }
- LOG.debug("** invokable --> {}", invokable);
- } else {
- LOG.debug("Skipping constructor with wrong number of arguments.");
- }
- }
- if (eligibleCount > 1) {
- LOG.warn("Found multiple invokable constructors for class {}, given arguments {}. Using the last one found.",
- target, args);
- }
- return retval;
- }
-
-
- public static void invokeConfigMethods(ObjectDef bean, Object instance, ExecutionContext context)
- throws InvocationTargetException, IllegalAccessException {
-
- List<ConfigMethodDef> methodDefs = bean.getConfigMethods();
- if(methodDefs == null || methodDefs.size() == 0){
- return;
- }
- Class clazz = instance.getClass();
- for(ConfigMethodDef methodDef : methodDefs){
- List<Object> args = methodDef.getArgs();
- if (args == null){
- args = new ArrayList();
- }
- if(methodDef.hasReferences()){
- args = resolveReferences(args, context);
- }
- String methodName = methodDef.getName();
- Method method = findCompatibleMethod(args, clazz, methodName);
- if(method != null) {
- Object[] methodArgs = getArgsWithListCoercian(args, method.getParameterTypes());
- method.invoke(instance, methodArgs);
- } else {
- String msg = String.format("Unable to find configuration method '%s' in class '%s' with arguments %s.",
- new Object[]{methodName, clazz.getName(), args});
- throw new IllegalArgumentException(msg);
- }
- }
- }
-
- private static Method findCompatibleMethod(List<Object> args, Class target, String methodName){
- Method retval = null;
- int eligibleCount = 0;
-
- LOG.debug("Target class: {}, methodName: {}, args: {}", target.getName(), methodName, args);
- Method[] methods = target.getMethods();
-
- for (Method method : methods) {
- Class[] paramClasses = method.getParameterTypes();
- if (paramClasses.length == args.size() && method.getName().equals(methodName)) {
- LOG.debug("found constructor with same number of args..");
- boolean invokable = false;
- if (args.size() == 0){
- // it's a method with zero args
- invokable = true;
- } else {
- invokable = canInvokeWithArgs(args, method.getParameterTypes());
- }
- if (invokable) {
- retval = method;
- eligibleCount++;
- }
- LOG.debug("** invokable --> {}", invokable);
- } else {
- LOG.debug("Skipping method with wrong number of arguments.");
- }
- }
- if (eligibleCount > 1) {
- LOG.warn("Found multiple invokable methods for class {}, method {}, given arguments {}. " +
- "Using the last one found.",
- new Object[]{target, methodName, args});
- }
- return retval;
- }
-
- /**
- * Given a java.util.List of contructor/method arguments, and a list of parameter types, attempt to convert the
- * list to an java.lang.Object array that can be used to invoke the constructor. If an argument needs
- * to be coerced from a List to an Array, do so.
- */
- private static Object[] getArgsWithListCoercian(List<Object> args, Class[] parameterTypes) {
-// Class[] parameterTypes = constructor.getParameterTypes();
- if (parameterTypes.length != args.size()) {
- throw new IllegalArgumentException("Contructor parameter count does not egual argument size.");
- }
- Object[] constructorParams = new Object[args.size()];
-
- // loop through the arguments, if we hit a list that has to be convered to an array,
- // perform the conversion
- for (int i = 0; i < args.size(); i++) {
- Object obj = args.get(i);
- Class paramType = parameterTypes[i];
- Class objectType = obj.getClass();
- LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
- paramType, objectType);
- if (paramType.equals(objectType)) {
- LOG.debug("They are the same class.");
- constructorParams[i] = args.get(i);
- continue;
- }
- if (paramType.isAssignableFrom(objectType)) {
- LOG.debug("Assignment is possible.");
- constructorParams[i] = args.get(i);
- continue;
- }
- if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
- LOG.debug("Its a primitive boolean.");
- Boolean bool = (Boolean)args.get(i);
- constructorParams[i] = bool.booleanValue();
- continue;
- }
- if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
- LOG.debug("Its a primitive number.");
- Number num = (Number)args.get(i);
- if(paramType == Float.TYPE){
- constructorParams[i] = num.floatValue();
- } else if (paramType == Double.TYPE) {
- constructorParams[i] = num.doubleValue();
- } else if (paramType == Long.TYPE) {
- constructorParams[i] = num.longValue();
- } else if (paramType == Integer.TYPE) {
- constructorParams[i] = num.intValue();
- } else if (paramType == Short.TYPE) {
- constructorParams[i] = num.shortValue();
- } else if (paramType == Byte.TYPE) {
- constructorParams[i] = num.byteValue();
- } else {
- constructorParams[i] = args.get(i);
- }
- continue;
- }
-
- // enum conversion
- if(paramType.isEnum() && objectType.equals(String.class)){
- LOG.debug("Yes, will convert a String to enum");
- constructorParams[i] = Enum.valueOf(paramType, (String)args.get(i));
- continue;
- }
-
- // List to array conversion
- if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
- // TODO more collection content type checking
- LOG.debug("Conversion appears possible...");
- List list = (List) obj;
- LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), list.get(0).getClass());
-
- // create an array of the right type
- Object newArrayObj = Array.newInstance(paramType.getComponentType(), list.size());
- for (int j = 0; j < list.size(); j++) {
- Array.set(newArrayObj, j, list.get(j));
-
- }
- constructorParams[i] = newArrayObj;
- LOG.debug("After conversion: {}", constructorParams[i]);
- }
- }
- return constructorParams;
- }
-
-
- /**
- * Determine if the given constructor/method parameter types are compatible given arguments List. Consider if
- * list coercian can make it possible.
- *
- * @param args
- * @param parameterTypes
- * @return
- */
- private static boolean canInvokeWithArgs(List<Object> args, Class[] parameterTypes) {
- if (parameterTypes.length != args.size()) {
- LOG.warn("parameter types were the wrong size");
- return false;
- }
-
- for (int i = 0; i < args.size(); i++) {
- Object obj = args.get(i);
- if (obj == null) {
- throw new IllegalArgumentException("argument shouldn't be null - index: " + i);
- }
- Class paramType = parameterTypes[i];
- Class objectType = obj.getClass();
- LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
- paramType, objectType);
- if (paramType.equals(objectType)) {
- LOG.debug("Yes, they are the same class.");
- } else if (paramType.isAssignableFrom(objectType)) {
- LOG.debug("Yes, assignment is possible.");
- } else if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
- LOG.debug("Yes, assignment is possible.");
- } else if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
- LOG.debug("Yes, assignment is possible.");
- } else if(paramType.isEnum() && objectType.equals(String.class)){
- LOG.debug("Yes, will convert a String to enum");
- } else if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
- // TODO more collection content type checking
- LOG.debug("Assignment is possible if we convert a List to an array.");
- LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), ((List) obj).get(0).getClass());
- } else {
- return false;
- }
- }
- return true;
- }
-
- public static boolean isPrimitiveNumber(Class clazz){
- return clazz.isPrimitive() && !clazz.equals(boolean.class);
- }
-
- public static boolean isPrimitiveBoolean(Class clazz){
- return clazz.isPrimitive() && clazz.equals(boolean.class);
- }
-}
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
deleted file mode 100644
index 2777854..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.api;
-
-
-import org.apache.storm.generated.StormTopology;
-
-import java.util.Map;
-
-/**
- * Marker interface for objects that can produce `StormTopology` objects.
- *
- * If a `topology-source` class implements the `getTopology()` method, Flux will
- * call that method. Otherwise, it will introspect the given class and look for a
- * similar method that produces a `StormTopology` instance.
- *
- * Note that it is not strictly necessary for a class to implement this interface.
- * If a class defines a method with a similar signature, Flux should be able to find
- * and invoke it.
- *
- */
-public interface TopologySource {
- public StormTopology getTopology(Map<String, Object> config);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
deleted file mode 100644
index 72ca5ae..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A representation of a Java object that is uniquely identifyable, and given a className, constructor arguments,
- * and properties, can be instantiated.
- */
-public class BeanDef extends ObjectDef {
- private String id;
-
- public String getId() {
- return id;
- }
-
- public void setId(String id) {
- this.id = id;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
deleted file mode 100644
index 652210c..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import java.util.List;
-
-/**
- * A bean list reference is a list of bean reference.
- */
-public class BeanListReference {
- public List<String> ids;
-
- public BeanListReference(){}
-
- public BeanListReference(List<String> ids){
- this.ids = ids;
- }
-
- public List<String> getIds() {
- return ids;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
deleted file mode 100644
index bd236f1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * A bean reference is simply a string pointer to another id.
- */
-public class BeanReference {
- public String id;
-
- public BeanReference(){}
-
- public BeanReference(String id){
- this.id = id;
- }
-
- public String getId() {
- return id;
- }
-
- public void setId(String id) {
- this.id = id;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
deleted file mode 100644
index 362abf1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * Bean representation of a Storm bolt.
- */
-public class BoltDef extends VertexDef {
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
deleted file mode 100644
index 69cabc3..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ConfigMethodDef {
- private String name;
- private List<Object> args;
- private boolean hasReferences = false;
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public List<Object> getArgs() {
- return args;
- }
-
- public void setArgs(List<Object> args) {
-
- List<Object> newVal = new ArrayList<Object>();
- for(Object obj : args){
- if(obj instanceof LinkedHashMap){
- Map map = (Map)obj;
- if(map.containsKey("ref") && map.size() == 1){
- newVal.add(new BeanReference((String)map.get("ref")));
- this.hasReferences = true;
- } else if (map.containsKey("reflist") && map.size() == 1) {
- newVal.add(new BeanListReference((List<String>) map.get("reflist")));
- this.hasReferences = true;
- } else {
- newVal.add(obj);
- }
- } else {
- newVal.add(obj);
- }
- }
- this.args = newVal;
- }
-
- public boolean hasReferences(){
- return this.hasReferences;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
deleted file mode 100644
index 1520006..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import org.apache.storm.Config;
-import org.apache.storm.task.IBolt;
-import org.apache.storm.topology.IRichSpout;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Container for all the objects required to instantiate a topology.
- */
-public class ExecutionContext {
- // parsed Topology definition
- TopologyDef topologyDef;
-
- // Storm config
- private Config config;
-
- // components
- private List<Object> compontents;
- // indexed by id
- private Map<String, Object> componentMap = new HashMap<String, Object>();
-
- private Map<String, IRichSpout> spoutMap = new HashMap<String, IRichSpout>();
-
- private List<IBolt> bolts;
- private Map<String, Object> boltMap = new HashMap<String, Object>();
-
- public ExecutionContext(TopologyDef topologyDef, Config config){
- this.topologyDef = topologyDef;
- this.config = config;
- }
-
- public TopologyDef getTopologyDef(){
- return this.topologyDef;
- }
-
- public void addSpout(String id, IRichSpout spout){
- this.spoutMap.put(id, spout);
- }
-
- public void addBolt(String id, Object bolt){
- this.boltMap.put(id, bolt);
- }
-
- public Object getBolt(String id){
- return this.boltMap.get(id);
- }
-
- public void addComponent(String id, Object value){
- this.componentMap.put(id, value);
- }
-
- public Object getComponent(String id){
- return this.componentMap.get(id);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
deleted file mode 100644
index e4fac8e..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import java.util.List;
-
-/**
- * Bean representation of a Storm stream grouping.
- */
-public class GroupingDef {
-
- /**
- * Types of stream groupings Storm allows
- */
- public static enum Type {
- ALL,
- CUSTOM,
- DIRECT,
- SHUFFLE,
- LOCAL_OR_SHUFFLE,
- FIELDS,
- GLOBAL,
- NONE
- }
-
- private Type type;
- private String streamId;
- private List<String> args;
- private ObjectDef customClass;
-
- public List<String> getArgs() {
- return args;
- }
-
- public void setArgs(List<String> args) {
- this.args = args;
- }
-
- public Type getType() {
- return type;
- }
-
- public void setType(Type type) {
- this.type = type;
- }
-
- public String getStreamId() {
- return streamId;
- }
-
- public void setStreamId(String streamId) {
- this.streamId = streamId;
- }
-
- public ObjectDef getCustomClass() {
- return customClass;
- }
-
- public void setCustomClass(ObjectDef customClass) {
- this.customClass = customClass;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
deleted file mode 100644
index 23fd9d2..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * Represents an include. Includes can be either a file or a classpath resource.
- *
- * If an include is marked as `override=true` then existing properties will be replaced.
- *
- */
-public class IncludeDef {
- private boolean resource = false;
- boolean override = false;
- private String file;
-
- public boolean isResource() {
- return resource;
- }
-
- public void setResource(boolean resource) {
- this.resource = resource;
- }
-
- public String getFile() {
- return file;
- }
-
- public void setFile(String file) {
- this.file = file;
- }
-
- public boolean isOverride() {
- return override;
- }
-
- public void setOverride(boolean override) {
- this.override = override;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
deleted file mode 100644
index 04a7e8a..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import org.apache.storm.Config;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A representation of a Java object that given a className, constructor arguments,
- * and properties, can be instantiated.
- */
-public class ObjectDef {
- private String className;
- private List<Object> constructorArgs;
- private boolean hasReferences;
- private List<PropertyDef> properties;
- private List<ConfigMethodDef> configMethods;
-
- public String getClassName() {
- return className;
- }
-
- public void setClassName(String className) {
- this.className = className;
- }
-
- public List<Object> getConstructorArgs() {
- return constructorArgs;
- }
-
- public void setConstructorArgs(List<Object> constructorArgs) {
-
- List<Object> newVal = new ArrayList<Object>();
- for(Object obj : constructorArgs){
- if(obj instanceof LinkedHashMap){
- Map map = (Map)obj;
- if(map.containsKey("ref") && map.size() == 1) {
- newVal.add(new BeanReference((String) map.get("ref")));
- this.hasReferences = true;
- } else if (map.containsKey("reflist") && map.size() == 1) {
- newVal.add(new BeanListReference((List<String>) map.get("reflist")));
- this.hasReferences = true;
- } else {
- newVal.add(obj);
- }
- } else {
- newVal.add(obj);
- }
- }
- this.constructorArgs = newVal;
- }
-
- public boolean hasConstructorArgs(){
- return this.constructorArgs != null && this.constructorArgs.size() > 0;
- }
-
- public boolean hasReferences(){
- return this.hasReferences;
- }
-
- public List<PropertyDef> getProperties() {
- return properties;
- }
-
- public void setProperties(List<PropertyDef> properties) {
- this.properties = properties;
- }
-
- public List<ConfigMethodDef> getConfigMethods() {
- return configMethods;
- }
-
- public void setConfigMethods(List<ConfigMethodDef> configMethods) {
- this.configMethods = configMethods;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
deleted file mode 100644
index f3d7704..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-public class PropertyDef {
- private String name;
- private Object value;
- private String ref;
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public Object getValue() {
- return value;
- }
-
- public void setValue(Object value) {
- if(this.ref != null){
- throw new IllegalStateException("A property can only have a value OR a reference, not both.");
- }
- this.value = value;
- }
-
- public String getRef() {
- return ref;
- }
-
- public void setRef(String ref) {
- if(this.value != null){
- throw new IllegalStateException("A property can only have a value OR a reference, not both.");
- }
- this.ref = ref;
- }
-
- public boolean isReference(){
- return this.ref != null;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
deleted file mode 100644
index 277c601..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * Bean representation of a Storm spout.
- */
-public class SpoutDef extends VertexDef {
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
deleted file mode 100644
index da80f1c..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * Represents a stream of tuples from one Storm component (Spout or Bolt) to another (an edge in the topology DAG).
- *
- * Required fields are `from` and `to`, which define the source and destination, and the stream `grouping`.
- *
- */
-public class StreamDef {
-
- private String name; // not used, placeholder for GUI, etc.
- private String from;
- private String to;
- private GroupingDef grouping;
-
- public String getTo() {
- return to;
- }
-
- public void setTo(String to) {
- this.to = to;
- }
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public String getFrom() {
- return from;
- }
-
- public void setFrom(String from) {
- this.from = from;
- }
-
- public GroupingDef getGrouping() {
- return grouping;
- }
-
- public void setGrouping(GroupingDef grouping) {
- this.grouping = grouping;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
deleted file mode 100644
index 86614f1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-
-/**
- * Bean represenation of a topology.
- *
- * It consists of the following:
- * 1. The topology name
- * 2. A `java.util.Map` representing the `org.apache.storm.config` for the topology
- * 3. A list of spout definitions
- * 4. A list of bolt definitions
- * 5. A list of stream definitions that define the flow between spouts and bolts.
- *
- */
-public class TopologyDef {
- private static Logger LOG = LoggerFactory.getLogger(TopologyDef.class);
-
- private String name;
- private Map<String, BeanDef> componentMap = new LinkedHashMap<String, BeanDef>(); // not required
- private List<IncludeDef> includes; // not required
- private Map<String, Object> config = new HashMap<String, Object>();
-
- // a "topology source" is a class that can produce a `StormTopology` thrift object.
- private TopologySourceDef topologySource;
-
- // the following are required if we're defining a core storm topology DAG in YAML, etc.
- private Map<String, BoltDef> boltMap = new LinkedHashMap<String, BoltDef>();
- private Map<String, SpoutDef> spoutMap = new LinkedHashMap<String, SpoutDef>();
- private List<StreamDef> streams = new ArrayList<StreamDef>();
-
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- public void setName(String name, boolean override){
- if(this.name == null || override){
- this.name = name;
- } else {
- LOG.warn("Ignoring attempt to set property 'name' with override == false.");
- }
- }
-
- public List<SpoutDef> getSpouts() {
- ArrayList<SpoutDef> retval = new ArrayList<SpoutDef>();
- retval.addAll(this.spoutMap.values());
- return retval;
- }
-
- public void setSpouts(List<SpoutDef> spouts) {
- this.spoutMap = new LinkedHashMap<String, SpoutDef>();
- for(SpoutDef spout : spouts){
- this.spoutMap.put(spout.getId(), spout);
- }
- }
-
- public List<BoltDef> getBolts() {
- ArrayList<BoltDef> retval = new ArrayList<BoltDef>();
- retval.addAll(this.boltMap.values());
- return retval;
- }
-
- public void setBolts(List<BoltDef> bolts) {
- this.boltMap = new LinkedHashMap<String, BoltDef>();
- for(BoltDef bolt : bolts){
- this.boltMap.put(bolt.getId(), bolt);
- }
- }
-
- public List<StreamDef> getStreams() {
- return streams;
- }
-
- public void setStreams(List<StreamDef> streams) {
- this.streams = streams;
- }
-
- public Map<String, Object> getConfig() {
- return config;
- }
-
- public void setConfig(Map<String, Object> config) {
- this.config = config;
- }
-
- public List<BeanDef> getComponents() {
- ArrayList<BeanDef> retval = new ArrayList<BeanDef>();
- retval.addAll(this.componentMap.values());
- return retval;
- }
-
- public void setComponents(List<BeanDef> components) {
- this.componentMap = new LinkedHashMap<String, BeanDef>();
- for(BeanDef component : components){
- this.componentMap.put(component.getId(), component);
- }
- }
-
- public List<IncludeDef> getIncludes() {
- return includes;
- }
-
- public void setIncludes(List<IncludeDef> includes) {
- this.includes = includes;
- }
-
- // utility methods
- public int parallelismForBolt(String boltId){
- return this.boltMap.get(boltId).getParallelism();
- }
-
- public BoltDef getBoltDef(String id){
- return this.boltMap.get(id);
- }
-
- public SpoutDef getSpoutDef(String id){
- return this.spoutMap.get(id);
- }
-
- public BeanDef getComponent(String id){
- return this.componentMap.get(id);
- }
-
- // used by includes implementation
- public void addAllBolts(List<BoltDef> bolts, boolean override){
- for(BoltDef bolt : bolts){
- String id = bolt.getId();
- if(this.boltMap.get(id) == null || override) {
- this.boltMap.put(bolt.getId(), bolt);
- } else {
- LOG.warn("Ignoring attempt to create bolt '{}' with override == false.", id);
- }
- }
- }
-
- public void addAllSpouts(List<SpoutDef> spouts, boolean override){
- for(SpoutDef spout : spouts){
- String id = spout.getId();
- if(this.spoutMap.get(id) == null || override) {
- this.spoutMap.put(spout.getId(), spout);
- } else {
- LOG.warn("Ignoring attempt to create spout '{}' with override == false.", id);
- }
- }
- }
-
- public void addAllComponents(List<BeanDef> components, boolean override) {
- for(BeanDef bean : components){
- String id = bean.getId();
- if(this.componentMap.get(id) == null || override) {
- this.componentMap.put(bean.getId(), bean);
- } else {
- LOG.warn("Ignoring attempt to create component '{}' with override == false.", id);
- }
- }
- }
-
- public void addAllStreams(List<StreamDef> streams, boolean override) {
- //TODO figure out how we want to deal with overrides. Users may want to add streams even when overriding other
- // properties. For now we just add them blindly which could lead to a potentially invalid topology.
- this.streams.addAll(streams);
- }
-
- public TopologySourceDef getTopologySource() {
- return topologySource;
- }
-
- public void setTopologySource(TopologySourceDef topologySource) {
- this.topologySource = topologySource;
- }
-
- public boolean isDslTopology(){
- return this.topologySource == null;
- }
-
-
- public boolean validate(){
- boolean hasSpouts = this.spoutMap != null && this.spoutMap.size() > 0;
- boolean hasBolts = this.boltMap != null && this.boltMap.size() > 0;
- boolean hasStreams = this.streams != null && this.streams.size() > 0;
- boolean hasSpoutsBoltsStreams = hasStreams && hasBolts && hasSpouts;
- // you cant define a topologySource and a DSL topology at the same time...
- if (!isDslTopology() && ((hasSpouts || hasBolts || hasStreams))) {
- return false;
- }
- if(isDslTopology() && (hasSpouts && hasBolts && hasStreams)) {
- return true;
- }
- return true;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
deleted file mode 100644
index d6a2f57..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-public class TopologySourceDef extends ObjectDef {
- public static final String DEFAULT_METHOD_NAME = "getTopology";
-
- private String methodName;
-
- public TopologySourceDef(){
- this.methodName = DEFAULT_METHOD_NAME;
- }
-
- public String getMethodName() {
- return methodName;
- }
-
- public void setMethodName(String methodName) {
- this.methodName = methodName;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
deleted file mode 100644
index e71bcc2..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.model;
-
-/**
- * Abstract parent class of component definitions
- * (spouts/bolts)
- */
-public abstract class VertexDef extends BeanDef {
-
- // default parallelism to 1 so if it's ommitted, the topology will still function.
- private int parallelism = 1;
-
- public int getParallelism() {
- return parallelism;
- }
-
- public void setParallelism(int parallelism) {
- this.parallelism = parallelism;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
deleted file mode 100644
index 2a18474..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.parser;
-
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.model.BoltDef;
-import org.apache.storm.flux.model.IncludeDef;
-import org.apache.storm.flux.model.SpoutDef;
-import org.apache.storm.flux.model.TopologyDef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.yaml.snakeyaml.TypeDescription;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-import java.io.ByteArrayOutputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.Map;
-import java.util.Properties;
-
-public class FluxParser {
- private static final Logger LOG = LoggerFactory.getLogger(FluxParser.class);
-
- private FluxParser(){}
-
- // TODO refactor input stream processing (see parseResource() method).
- public static TopologyDef parseFile(String inputFile, boolean dumpYaml, boolean processIncludes,
- String propertiesFile, boolean envSub) throws IOException {
-
- FileInputStream in = new FileInputStream(inputFile);
- TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
- in.close();
-
- return topology;
- }
-
- public static TopologyDef parseResource(String resource, boolean dumpYaml, boolean processIncludes,
- String propertiesFile, boolean envSub) throws IOException {
-
- InputStream in = FluxParser.class.getResourceAsStream(resource);
- TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
- in.close();
-
- return topology;
- }
-
- public static TopologyDef parseInputStream(InputStream inputStream, boolean dumpYaml, boolean processIncludes,
- String propertiesFile, boolean envSub) throws IOException {
-
- Yaml yaml = yaml();
-
- if (inputStream == null) {
- LOG.error("Unable to load input stream");
- System.exit(1);
- }
-
- TopologyDef topology = loadYaml(yaml, inputStream, propertiesFile, envSub);
-
- if (dumpYaml) {
- dumpYaml(topology, yaml);
- }
-
- if (processIncludes) {
- return processIncludes(yaml, topology, propertiesFile, envSub);
- } else {
- return topology;
- }
- }
-
- private static TopologyDef loadYaml(Yaml yaml, InputStream in, String propsFile, boolean envSubstitution) throws IOException {
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- LOG.info("loading YAML from input stream...");
- int b = -1;
- while((b = in.read()) != -1){
- bos.write(b);
- }
-
- // TODO substitution implementation is not exactly efficient or kind to memory...
- String str = bos.toString();
- // properties file substitution
- if(propsFile != null){
- LOG.info("Performing property substitution.");
- InputStream propsIn = new FileInputStream(propsFile);
- Properties props = new Properties();
- props.load(propsIn);
- for(Object key : props.keySet()){
- str = str.replace("${" + key + "}", props.getProperty((String)key));
- }
- } else {
- LOG.info("Not performing property substitution.");
- }
-
- // environment variable substitution
- if(envSubstitution){
- LOG.info("Performing environment variable substitution...");
- Map<String, String> envs = System.getenv();
- for(String key : envs.keySet()){
- str = str.replace("${ENV-" + key + "}", envs.get(key));
- }
- } else {
- LOG.info("Not performing environment variable substitution.");
- }
- return (TopologyDef)yaml.load(str);
- }
-
- private static void dumpYaml(TopologyDef topology, Yaml yaml){
- System.out.println("Configuration (interpreted): \n" + yaml.dump(topology));
- }
-
- private static Yaml yaml(){
- Constructor constructor = new Constructor(TopologyDef.class);
-
- TypeDescription topologyDescription = new TypeDescription(TopologyDef.class);
- topologyDescription.putListPropertyType("spouts", SpoutDef.class);
- topologyDescription.putListPropertyType("bolts", BoltDef.class);
- topologyDescription.putListPropertyType("includes", IncludeDef.class);
- constructor.addTypeDescription(topologyDescription);
-
- Yaml yaml = new Yaml(constructor);
- return yaml;
- }
-
- /**
- *
- * @param yaml the yaml parser for parsing the include file(s)
- * @param topologyDef the topology definition containing (possibly zero) includes
- * @return The TopologyDef with includes resolved.
- */
- private static TopologyDef processIncludes(Yaml yaml, TopologyDef topologyDef, String propsFile, boolean envSub)
- throws IOException {
- //TODO support multiple levels of includes
- if(topologyDef.getIncludes() != null) {
- for (IncludeDef include : topologyDef.getIncludes()){
- TopologyDef includeTopologyDef = null;
- if (include.isResource()) {
- LOG.info("Loading includes from resource: {}", include.getFile());
- includeTopologyDef = parseResource(include.getFile(), true, false, propsFile, envSub);
- } else {
- LOG.info("Loading includes from file: {}", include.getFile());
- includeTopologyDef = parseFile(include.getFile(), true, false, propsFile, envSub);
- }
-
- // if overrides are disabled, we won't replace anything that already exists
- boolean override = include.isOverride();
- // name
- if(includeTopologyDef.getName() != null){
- topologyDef.setName(includeTopologyDef.getName(), override);
- }
-
- // config
- if(includeTopologyDef.getConfig() != null) {
- //TODO move this logic to the model class
- Map<String, Object> config = topologyDef.getConfig();
- Map<String, Object> includeConfig = includeTopologyDef.getConfig();
- if(override) {
- config.putAll(includeTopologyDef.getConfig());
- } else {
- for(String key : includeConfig.keySet()){
- if(config.containsKey(key)){
- LOG.warn("Ignoring attempt to set topology config property '{}' with override == false", key);
- }
- else {
- config.put(key, includeConfig.get(key));
- }
- }
- }
- }
-
- //component overrides
- if(includeTopologyDef.getComponents() != null){
- topologyDef.addAllComponents(includeTopologyDef.getComponents(), override);
- }
- //bolt overrides
- if(includeTopologyDef.getBolts() != null){
- topologyDef.addAllBolts(includeTopologyDef.getBolts(), override);
- }
- //spout overrides
- if(includeTopologyDef.getSpouts() != null) {
- topologyDef.addAllSpouts(includeTopologyDef.getSpouts(), override);
- }
- //stream overrides
- //TODO streams should be uniquely identifiable
- if(includeTopologyDef.getStreams() != null) {
- topologyDef.addAllStreams(includeTopologyDef.getStreams(), override);
- }
- } // end include processing
- }
- return topologyDef;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/resources/splash.txt
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/resources/splash.txt b/external/flux/flux-core/src/main/resources/splash.txt
deleted file mode 100644
index 337931a..0000000
--- a/external/flux/flux-core/src/main/resources/splash.txt
+++ /dev/null
@@ -1,9 +0,0 @@
-\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557 \u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557
-\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
-\u2588\u2588\u2588\u2588\u2588\u2557 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
-\u2588\u2588\u2554\u2550\u2550\u255d \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
-\u2588\u2588\u2551 \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
-\u255a\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d \u255a\u2550\u255d
-+- Apache Storm -+
-+- data FLow User eXperience -+
-Version: ${project.version}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
deleted file mode 100644
index ff67867..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-public class FluxBuilderTest {
-
- @Test
- public void testIsPrimitiveNumber() throws Exception {
- assertTrue(FluxBuilder.isPrimitiveNumber(int.class));
- assertFalse(FluxBuilder.isPrimitiveNumber(boolean.class));
- assertFalse(FluxBuilder.isPrimitiveNumber(String.class));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
deleted file mode 100644
index c5807f8..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux;
-
-import org.junit.Test;
-
-public class IntegrationTest {
-
- private static boolean skipTest = true;
-
- static {
- String skipStr = System.getProperty("skipIntegration");
- if(skipStr != null && skipStr.equalsIgnoreCase("false")){
- skipTest = false;
- }
- }
-
- @Test
- public void testRunTopologySource() throws Exception {
- if(!skipTest) {
- Flux.main(new String[]{"-s", "30000", "src/test/resources/configs/existing-topology.yaml"});
- }
- }
-}
[12/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
deleted file mode 100644
index a373483..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-public class StormSqlFunctions {
- public static Boolean eq(Object b0, Object b1) {
- if (b0 == null || b1 == null) {
- return null;
- }
- return b0.equals(b1);
- }
-
- public static Boolean ne(Object b0, Object b1) {
- if (b0 == null || b1 == null) {
- return null;
- }
- return !b0.equals(b1);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
deleted file mode 100644
index e78f354..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.calcite;
-
-import org.apache.calcite.interpreter.Context;
-
-public class DebuggableExecutableExpression implements ExecutableExpression {
- private ExecutableExpression delegate;
- private String delegateCode;
-
- public DebuggableExecutableExpression(ExecutableExpression delegate, String delegateCode) {
- this.delegate = delegate;
- this.delegateCode = delegateCode;
- }
-
- @Override
- public Object execute(Context context) {
- return delegate.execute(context);
- }
-
- @Override
- public void execute(Context context, Object[] results) {
- delegate.execute(context, results);
- }
-
- public String getDelegateCode() {
- return delegateCode;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
deleted file mode 100644
index 8416945..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.calcite;
-
-import org.apache.calcite.interpreter.Context;
-
-import java.io.Serializable;
-
-/**
- * Compiled executable expression.
- */
-public interface ExecutableExpression extends Serializable {
- Object execute(Context context);
- void execute(Context context, Object[] results);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
deleted file mode 100644
index 4861b43..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.calcite;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.runtime.Hook;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.util.Holder;
-
-import java.io.Serializable;
-import java.util.Calendar;
-import java.util.TimeZone;
-
-/**
- * This is based on SlimDataContext in Calcite, and borrow some from DataContextImpl in Calcite.
- */
-public class StormDataContext implements DataContext, Serializable {
- private final ImmutableMap<Object, Object> map;
-
- public StormDataContext() {
- // Store the time at which the query started executing. The SQL
- // standard says that functions such as CURRENT_TIMESTAMP return the
- // same value throughout the query.
-
- final Holder<Long> timeHolder = Holder.of(System.currentTimeMillis());
-
- // Give a hook chance to alter the clock.
- Hook.CURRENT_TIME.run(timeHolder);
- final long time = timeHolder.get();
- final TimeZone timeZone = Calendar.getInstance().getTimeZone();
- final long localOffset = timeZone.getOffset(time);
- final long currentOffset = localOffset;
-
- ImmutableMap.Builder<Object, Object> builder = ImmutableMap.builder();
- builder.put(Variable.UTC_TIMESTAMP.camelName, time)
- .put(Variable.CURRENT_TIMESTAMP.camelName, time + currentOffset)
- .put(Variable.LOCAL_TIMESTAMP.camelName, time + localOffset)
- .put(Variable.TIME_ZONE.camelName, timeZone);
- map = builder.build();
- }
-
- @Override
- public SchemaPlus getRootSchema() {
- return null;
- }
-
- @Override
- public JavaTypeFactory getTypeFactory() {
- return null;
- }
-
- @Override
- public QueryProvider getQueryProvider() {
- return null;
- }
-
- @Override
- public Object get(String name) {
- return map.get(name);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
deleted file mode 100644
index d81e772..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.datasource.socket;
-
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
-import org.apache.storm.sql.runtime.datasource.socket.trident.TridentSocketSpout;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a Socket data source based on the URI and properties. The URI has the format of
- * socket://[host]:[port]. Both of host and port are mandatory.
- *
- * Note that it connects to given host and port, and receive the message if it's used for input source,
- * and send the message if it's used for output data source.
- */
-public class SocketDataSourcesProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "socket";
- }
-
- private static class SocketTridentDataSource implements ISqlTridentDataSource {
-
- private final String host;
- private final int port;
- private final Scheme scheme;
- private final IOutputSerializer serializer;
-
- SocketTridentDataSource(Scheme scheme, IOutputSerializer serializer, String host, int port) {
- this.scheme = scheme;
- this.serializer = serializer;
- this.host = host;
- this.port = port;
- }
-
- @Override
- public ITridentDataSource getProducer() {
- return new TridentSocketSpout(scheme, host, port);
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- StateFactory stateFactory = new SocketState.Factory(host, port);
- StateUpdater<SocketState> stateUpdater = new SocketStateUpdater(serializer);
- return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
- }
- }
-
- @Override
- public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) {
- String host = uri.getHost();
- int port = uri.getPort();
- if (port == -1) {
- throw new RuntimeException("Port information is not available. URI: " + uri);
- }
-
- List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
- Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
- IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-
- return new SocketTridentDataSource(scheme, serializer, host, port);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
deleted file mode 100644
index 3f85756..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.datasource.socket.trident;
-
-import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.trident.state.State;
-import org.apache.storm.trident.state.StateFactory;
-
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.net.Socket;
-import java.util.Map;
-
-/**
- * Trident State implementation of Socket. It only supports writing.
- */
-public class SocketState implements State {
- /**
- * {@inheritDoc}
- */
- @Override
- public void beginCommit(Long txid) {
- // no-op
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void commit(Long txid) {
- // no-op
- }
-
- public static class Factory implements StateFactory {
- private final String host;
- private final int port;
-
- public Factory(String host, int port) {
- this.host = host;
- this.port = port;
- }
-
- @Override
- public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
- BufferedWriter out;
- try {
- Socket socket = new Socket(host, port);
- out = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream()));
- } catch (IOException e) {
- throw new RuntimeException("Exception while initializing socket for State. host " +
- host + " port " + port, e);
- }
-
- // State doesn't have close() and Storm actually doesn't guarantee so we can't release socket resource anyway
- return new SocketState(out);
- }
- }
-
- private BufferedWriter out;
-
- private SocketState(BufferedWriter out) {
- this.out = out;
- }
-
- public void write(String str) throws IOException {
- out.write(str);
- }
-
- public void flush() throws IOException {
- out.flush();
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
deleted file mode 100644
index 3062a90..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.datasource.socket.trident;
-
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.state.BaseStateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * StateUpdater for SocketState. Serializes tuple one by one and writes to socket, and finally flushes.
- */
-public class SocketStateUpdater extends BaseStateUpdater<SocketState> {
- private static final Logger LOG = LoggerFactory.getLogger(SocketStateUpdater.class);
-
- private final IOutputSerializer outputSerializer;
-
- public SocketStateUpdater(IOutputSerializer outputSerializer) {
- this.outputSerializer = outputSerializer;
- }
-
- @Override
- public void updateState(SocketState state, List<TridentTuple> tuples, TridentCollector collector) {
- try {
- for (TridentTuple tuple : tuples) {
- byte[] array = outputSerializer.write(tuple.getValues(), null).array();
- String data = new String(array);
- state.write(data + "\n");
- }
- state.flush();
- } catch (IOException e) {
- LOG.error("Error while updating state.", e);
- collector.reportError(e);
- throw new RuntimeException("Error while updating state.", e);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
deleted file mode 100644
index 97f63a7..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.runtime.datasource.socket.trident;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.storm.Config;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.spout.IBatchSpout;
-import org.apache.storm.tuple.Fields;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingDeque;
-import java.util.concurrent.LinkedBlockingDeque;
-
-/**
- * Trident Spout for Socket data. Only available for Storm SQL, and only use for test purposes.
- */
-public class TridentSocketSpout implements IBatchSpout {
- private static final Logger LOG = LoggerFactory.getLogger(TridentSocketSpout.class);
-
- private final String host;
- private final int port;
- private final Scheme scheme;
-
- private volatile boolean _running = true;
-
- private BlockingDeque<String> queue;
- private Socket socket;
- private Thread readerThread;
- private BufferedReader in;
- private ObjectMapper objectMapper;
-
- private Map<Long, List<List<Object>>> batches;
-
- public TridentSocketSpout(Scheme scheme, String host, int port) {
- this.scheme = scheme;
- this.host = host;
- this.port = port;
- }
-
- @Override
- public void open(Map conf, TopologyContext context) {
- this.queue = new LinkedBlockingDeque<>();
- this.objectMapper = new ObjectMapper();
- this.batches = new HashMap<>();
-
- try {
- socket = new Socket(host, port);
- in = new BufferedReader(new InputStreamReader(socket.getInputStream()));
- } catch (IOException e) {
- throw new RuntimeException("Error opening socket: host " + host + " port " + port);
- }
-
- readerThread = new Thread(new SocketReaderRunnable());
- readerThread.start();
- }
-
- @Override
- public void emitBatch(long batchId, TridentCollector collector) {
- // read line and parse it to json
- List<List<Object>> batch = this.batches.get(batchId);
- if (batch == null) {
- batch = new ArrayList<>();
-
- while(queue.peek() != null) {
- String line = queue.poll();
- List<Object> values = convertLineToTuple(line);
- if (values == null) {
- continue;
- }
-
- batch.add(values);
- }
-
- this.batches.put(batchId, batch);
- }
-
- for (List<Object> list : batch) {
- collector.emit(list);
- }
- }
-
- private List<Object> convertLineToTuple(String line) {
- return scheme.deserialize(ByteBuffer.wrap(line.getBytes()));
- }
-
- @Override
- public void ack(long batchId) {
- this.batches.remove(batchId);
- }
-
- @Override
- public void close() {
- _running = false;
- readerThread.interrupt();
- queue.clear();
-
- closeQuietly(in);
- closeQuietly(socket);
- }
-
- @Override
- public Map<String, Object> getComponentConfiguration() {
- Config conf = new Config();
- conf.setMaxTaskParallelism(1);
- return conf;
- }
-
- @Override
- public Fields getOutputFields() {
- return scheme.getOutputFields();
- }
-
- private class SocketReaderRunnable implements Runnable {
- public void run() {
- while (_running) {
- try {
- String line = in.readLine();
- if (line == null) {
- throw new RuntimeException("EOF reached from the socket. We can't read the data any more.");
- }
-
- queue.push(line.trim());
- } catch (Throwable t) {
- // This spout is added to test purpose, so just failing fast doesn't hurt much
- die(t);
- }
- }
- }
- }
-
- private void die(Throwable t) {
- LOG.error("Halting process: TridentSocketSpout died.", t);
- if (_running || (t instanceof Error)) { //don't exit if not running, unless it is an Error
- System.exit(11);
- }
- }
-
- private void closeQuietly(final Closeable closeable) {
- try {
- if (closeable != null) {
- closeable.close();
- }
- } catch (final IOException ioe) {
- // ignore
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
deleted file mode 100644
index 3bf1a23..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.avro;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * AvroScheme uses generic(without code generation) instead of specific(with code generation) readers.
- */
-public class AvroScheme implements Scheme {
- private final String schemaString;
- private final List<String> fieldNames;
- private final CachedSchemas schemas;
-
- public AvroScheme(String schemaString, List<String> fieldNames) {
- this.schemaString = schemaString;
- this.fieldNames = fieldNames;
- this.schemas = new CachedSchemas();
- }
-
- @Override
- public List<Object> deserialize(ByteBuffer ser) {
- try {
- Schema schema = schemas.getSchema(schemaString);
- DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
- BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(Utils.toByteArray(ser), null);
- GenericRecord record = reader.read(null, decoder);
-
- ArrayList<Object> list = new ArrayList<>(fieldNames.size());
- for (String field : fieldNames) {
- Object value = record.get(field);
- // Avro strings are stored using a special Avro Utf8 type instead of using Java primitives
- list.add(SerdeUtils.convertAvroUtf8(value));
- }
- return list;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public Fields getOutputFields() {
- return new Fields(fieldNames);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
deleted file mode 100644
index 5dc3393..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.avro;
-
-import com.google.common.base.Preconditions;
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.Encoder;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * AvroSerializer uses generic(without code generation) instead of specific(with code generation) writers.
- */
-public class AvroSerializer implements IOutputSerializer, Serializable {
- private final String schemaString;
- private final List<String> fieldNames;
- private final CachedSchemas schemas;
-
- public AvroSerializer(String schemaString, List<String> fieldNames) {
- this.schemaString = schemaString;
- this.fieldNames = fieldNames;
- this.schemas = new CachedSchemas();
- }
-
- @Override
- public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
- Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schemas");
- try {
- Schema schema = schemas.getSchema(schemaString);
- GenericRecord record = new GenericData.Record(schema);
- for (int i = 0; i < fieldNames.size(); i++) {
- record.put(fieldNames.get(i), data.get(i));
- }
-
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- DatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
- Encoder encoder = EncoderFactory.get().directBinaryEncoder(out, null);
- writer.write(record, encoder);
- encoder.flush();
- byte[] bytes = out.toByteArray();
- out.close();
- return ByteBuffer.wrap(bytes);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
deleted file mode 100644
index 4f0e747..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.avro;
-
-import org.apache.avro.Schema;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-// TODO this class is reserved for supporting messages with different schemas.
-// current only one schema in the cache
-public class CachedSchemas implements Serializable{
-
- private final Map<String, Schema> cache = new HashMap<>();
-
- public Schema getSchema(String schemaString) {
- Schema schema = cache.get(schemaString);
- if (schema == null) {
- schema = new Schema.Parser().parse(schemaString);
- cache.put(schemaString, schema);
- }
- return schema;
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
deleted file mode 100644
index 34fb1bb..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.csv;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVParser;
-import org.apache.commons.csv.CSVRecord;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * CsvScheme uses the standard RFC4180 CSV Parser
- * One of the difference from Tsv format is that fields with embedded commas will be quoted.
- * eg: a,"b,c",d is allowed.
- *
- * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
- */
-public class CsvScheme implements Scheme {
- private final List<String> fieldNames;
-
- public CsvScheme(List<String> fieldNames) {
- this.fieldNames = fieldNames;
- }
-
- @Override
- public List<Object> deserialize(ByteBuffer ser) {
- try {
- String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
- CSVParser parser = CSVParser.parse(data, CSVFormat.RFC4180);
- CSVRecord record = parser.getRecords().get(0);
- Preconditions.checkArgument(record.size() == fieldNames.size(), "Invalid schema");
-
- ArrayList<Object> list = new ArrayList<>(fieldNames.size());
- for (int i = 0; i < record.size(); i++) {
- list.add(record.get(i));
- }
- return list;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public Fields getOutputFields() {
- return new Fields(fieldNames);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
deleted file mode 100644
index 0d3bd74..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.csv;
-
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVPrinter;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.io.StringWriter;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-/**
- * CsvSerializer uses the standard RFC4180 CSV Parser
- * One of the difference from Tsv format is that fields with embedded commas will be quoted.
- * eg: a,"b,c",d is allowed.
- *
- * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
- */
-public class CsvSerializer implements IOutputSerializer, Serializable {
- private final List<String> fields; //reserved for future
-
- public CsvSerializer(List<String> fields) {
- this.fields = fields;
- }
-
- @Override
- public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
- try {
- StringWriter writer = new StringWriter();
- CSVPrinter printer = new CSVPrinter(writer, CSVFormat.RFC4180);
- for (Object o : data) {
- printer.print(o);
- }
- //since using StringWriter, we do not need to close it.
- return ByteBuffer.wrap(writer.getBuffer().toString().getBytes(StandardCharsets.UTF_8));
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
deleted file mode 100644
index d288fa1..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.json;
-
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-public class JsonScheme implements Scheme {
- private final List<String> fields;
-
- public JsonScheme(List<String> fields) {
- this.fields = fields;
- }
-
- @Override
- public List<Object> deserialize(ByteBuffer ser) {
- ObjectMapper mapper = new ObjectMapper();
- try {
- @SuppressWarnings("unchecked")
- HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
- ArrayList<Object> list = new ArrayList<>(fields.size());
- for (String f : fields) {
- list.add(map.get(f));
- }
- return list;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- @Override
- public Fields getOutputFields() {
- return new Fields(fields);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
deleted file mode 100644
index 1e825c4..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.json;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.google.common.base.Preconditions;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.io.StringWriter;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-public class JsonSerializer implements IOutputSerializer, Serializable {
- private final List<String> fieldNames;
- private final JsonFactory jsonFactory;
-
- public JsonSerializer(List<String> fieldNames) {
- this.fieldNames = fieldNames;
- jsonFactory = new JsonFactory();
- }
-
- @Override
- public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
- Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schema");
- StringWriter sw = new StringWriter();
- try (JsonGenerator jg = jsonFactory.createGenerator(sw)) {
- jg.writeStartObject();
- for (int i = 0; i < fieldNames.size(); ++i) {
- jg.writeFieldName(fieldNames.get(i));
- jg.writeObject(data.get(i));
- }
- jg.writeEndObject();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
deleted file mode 100644
index 310494c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.tsv;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * TsvScheme uses a simple delimited format implemention by splitting string,
- * and it supports user defined delimiter.
- */
-public class TsvScheme implements Scheme {
- private final List<String> fieldNames;
- private final char delimiter;
-
- public TsvScheme(List<String> fieldNames, char delimiter) {
- this.fieldNames = fieldNames;
- this.delimiter = delimiter;
- }
-
- @Override
- public List<Object> deserialize(ByteBuffer ser) {
- String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
- List<String> parts = org.apache.storm.sql.runtime.utils.Utils.split(data, delimiter);
- Preconditions.checkArgument(parts.size() == fieldNames.size(), "Invalid schema");
-
- ArrayList<Object> list = new ArrayList<>(fieldNames.size());
- list.addAll(parts);
- return list;
- }
-
- @Override
- public Fields getOutputFields() {
- return new Fields(fieldNames);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
deleted file mode 100644
index 1cf1c76..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.serde.tsv;
-
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-/**
- * TsvSerializer uses a simple delimited format implemention by splitting string,
- * and it supports user defined delimiter.
- */
-public class TsvSerializer implements IOutputSerializer, Serializable {
- private final List<String> fields; //reserved for future
- private final char delimiter;
-
- public TsvSerializer(List<String> fields, char delimiter) {
- this.fields = fields;
- this.delimiter = delimiter;
- }
-
- @Override
- public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
- StringBuilder sb = new StringBuilder(512); // 512: for most scenes to avoid inner array resizing
- for (int i = 0; i < data.size(); i++) {
- Object o = data.get(i);
- if (i == 0) {
- sb.append(o);
- } else {
- sb.append(delimiter);
- sb.append(o);
- }
- }
- return ByteBuffer.wrap(sb.toString().getBytes(StandardCharsets.UTF_8));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
deleted file mode 100644
index 6c76481..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.storm.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.OperationAwareFlatMapFunction;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.Map;
-
-public class EvaluationCalc implements OperationAwareFlatMapFunction {
- private static final Logger LOG = LoggerFactory.getLogger(EvaluationCalc.class);
-
- private final ExecutableExpression filterInstance;
- private final ExecutableExpression projectionInstance;
- private final Object[] outputValues;
- private final DataContext dataContext;
-
- public EvaluationCalc(ExecutableExpression filterInstance, ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
- this.filterInstance = filterInstance;
- this.projectionInstance = projectionInstance;
- this.outputValues = new Object[outputCount];
- this.dataContext = dataContext;
- }
-
- @Override
- public void prepare(Map conf, TridentOperationContext context) {
- if (projectionInstance != null && projectionInstance instanceof DebuggableExecutableExpression) {
- LOG.info("Expression code for projection: \n{}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
- }
- if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
- LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
- }
- }
-
- @Override
- public void cleanup() {
-
- }
-
- @Override
- public Iterable<Values> execute(TridentTuple input) {
- Context calciteContext = new StormContext(dataContext);
- calciteContext.values = input.getValues().toArray();
-
- if (filterInstance != null) {
- filterInstance.execute(calciteContext, outputValues);
- // filtered out
- if (outputValues[0] == null || !((Boolean) outputValues[0])) {
- return Collections.emptyList();
- }
- }
-
- if (projectionInstance != null) {
- projectionInstance.execute(calciteContext, outputValues);
- return Collections.singletonList(new Values(outputValues));
- } else {
- return Collections.singletonList(new Values(input.getValues()));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
deleted file mode 100644
index 9314852..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.storm.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.BaseFilter;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-public class EvaluationFilter extends BaseFilter {
- private static final Logger LOG = LoggerFactory.getLogger(EvaluationFilter.class);
-
- private final ExecutableExpression filterInstance;
- private final DataContext dataContext;
- private final Object[] outputValues;
-
- public EvaluationFilter(ExecutableExpression filterInstance, DataContext dataContext) {
- this.filterInstance = filterInstance;
- this.dataContext = dataContext;
- this.outputValues = new Object[1];
- }
-
- @Override
- public void prepare(Map conf, TridentOperationContext context) {
- if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
- LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
- }
- }
-
- @Override
- public boolean isKeep(TridentTuple tuple) {
- Context calciteContext = new StormContext(dataContext);
- calciteContext.values = tuple.getValues().toArray();
- filterInstance.execute(calciteContext, outputValues);
- return (outputValues[0] != null && (boolean) outputValues[0]);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
deleted file mode 100644
index 2608104..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.storm.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.OperationAwareMapFunction;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-public class EvaluationFunction implements OperationAwareMapFunction {
- private static final Logger LOG = LoggerFactory.getLogger(EvaluationFunction.class);
-
- private final ExecutableExpression projectionInstance;
- private final Object[] outputValues;
- private final DataContext dataContext;
-
- public EvaluationFunction(ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
- this.projectionInstance = projectionInstance;
- this.outputValues = new Object[outputCount];
- this.dataContext = dataContext;
- }
-
- @Override
- public void prepare(Map conf, TridentOperationContext context) {
- if (projectionInstance instanceof DebuggableExecutableExpression) {
- LOG.info("Expression code: {}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
- }
- }
-
- @Override
- public void cleanup() {
-
- }
-
- @Override
- public Values execute(TridentTuple input) {
- Context calciteContext = new StormContext(dataContext);
- calciteContext.values = input.getValues().toArray();
- projectionInstance.execute(calciteContext, outputValues);
- return new Values(outputValues);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
deleted file mode 100644
index 4c3a266..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.storm.sql.runtime.trident.functions;
-
-import org.apache.storm.trident.operation.BaseFunction;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public class ForwardFunction extends BaseFunction {
- @Override
- public void execute(TridentTuple tuple, TridentCollector collector) {
- collector.emit(tuple.getValues());
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
deleted file mode 100644
index efd5d25..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.utils;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.FieldInfo;
-
-import java.io.Serializable;
-import java.util.List;
-
-public final class FieldInfoUtils {
-
- public static List<String> getFieldNames(List<FieldInfo> fields) {
- return Lists.transform(fields, new FieldNameExtractor());
- }
-
- private static class FieldNameExtractor implements Function<FieldInfo, String>, Serializable {
- @Override
- public String apply(FieldInfo fieldInfo) {
- return fieldInfo.name();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
deleted file mode 100644
index 6b3dfc9..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.utils;
-
-import static org.apache.commons.lang.StringUtils.isNotEmpty;
-
-import com.google.common.base.Preconditions;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.util.Utf8;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
-import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
-import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
-import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
-import org.apache.storm.sql.runtime.serde.json.JsonScheme;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
-import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
-import org.apache.storm.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public final class SerdeUtils {
- public static Scheme getScheme(String inputFormatClass, Properties properties, List<String> fieldNames) {
- Scheme scheme;
- if (isNotEmpty(inputFormatClass)) {
- if (JsonScheme.class.getName().equals(inputFormatClass)) {
- scheme = new JsonScheme(fieldNames);
- } else if (TsvScheme.class.getName().equals(inputFormatClass)) {
- String delimiter = properties.getProperty("input.tsv.delimiter", "\t");
- scheme = new TsvScheme(fieldNames, delimiter.charAt(0));
- } else if (CsvScheme.class.getName().equals(inputFormatClass)) {
- scheme = new CsvScheme(fieldNames);
- } else if (AvroScheme.class.getName().equals(inputFormatClass)) {
- String schemaString = properties.getProperty("input.avro.schema");
- Preconditions.checkArgument(isNotEmpty(schemaString), "input.avro.schema can not be empty");
- scheme = new AvroScheme(schemaString, fieldNames);
- } else {
- scheme = Utils.newInstance(inputFormatClass);
- }
- } else {
- //use JsonScheme as the default scheme
- scheme = new JsonScheme(fieldNames);
- }
- return scheme;
- }
-
- public static IOutputSerializer getSerializer(String outputFormatClass, Properties properties, List<String> fieldNames) {
- IOutputSerializer serializer;
- if (isNotEmpty(outputFormatClass)) {
- if (JsonSerializer.class.getName().equals(outputFormatClass)) {
- serializer = new JsonSerializer(fieldNames);
- } else if (TsvSerializer.class.getName().equals(outputFormatClass)) {
- String delimiter = properties.getProperty("output.tsv.delimiter", "\t");
- serializer = new TsvSerializer(fieldNames, delimiter.charAt(0));
- } else if (CsvSerializer.class.getName().equals(outputFormatClass)) {
- serializer = new CsvSerializer(fieldNames);
- } else if (AvroSerializer.class.getName().equals(outputFormatClass)) {
- String schemaString = properties.getProperty("output.avro.schema");
- Preconditions.checkArgument(isNotEmpty(schemaString), "output.avro.schema can not be empty");
- serializer = new AvroSerializer(schemaString, fieldNames);
- } else {
- serializer = Utils.newInstance(outputFormatClass);
- }
- } else {
- //use JsonSerializer as the default serializer
- serializer = new JsonSerializer(fieldNames);
- }
- return serializer;
- }
-
- public static Object convertAvroUtf8(Object value){
- Object ret;
- if (value instanceof Utf8) {
- ret = value.toString();
- } else if (value instanceof Map<?, ?>) {
- ret = convertAvroUtf8Map((Map<Object,Object>)value);
- } else if (value instanceof GenericData.Array) {
- ret = convertAvroUtf8Array((GenericData.Array)value);
- } else {
- ret = value;
- }
- return ret;
- }
-
- public static Object convertAvroUtf8Map(Map<Object,Object> value) {
- Map<Object, Object> map = new HashMap<>(value.size());
- for (Map.Entry<Object, Object> entry : value.entrySet()) {
- Object k = convertAvroUtf8(entry.getKey());
- Object v = convertAvroUtf8(entry.getValue());
- map.put(k, v);
- }
- return map;
- }
-
- public static Object convertAvroUtf8Array(GenericData.Array value){
- List<Object> ls = new ArrayList<>(value.size());
- for(Object o : value){
- ls.add(convertAvroUtf8(o));
- }
- return ls;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
deleted file mode 100644
index a0f3af3..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime.utils;
-
-import java.util.LinkedList;
-import java.util.List;
-
-public final class Utils {
-
- /**
- * This method for splitting string into parts by a delimiter
- * It has higher performance than String.split(String regex)
- *
- * @param data
- * @param delimiter
- * @return
- */
- public static List<String> split(String data, char delimiter){
- List<String> list = new LinkedList<>();
- //do not use .toCharArray avoid system copy
- StringBuilder sb = new StringBuilder(512);
- int len = data.length();
- for (int i=0; i < len; i++) {
- char ch = data.charAt(i);
- if(ch == delimiter){
- list.add(sb.toString());
- sb.setLength(0);
- if(i == len - 1){
- list.add("");
- }
- }else{
- sb.append(ch);
- }
- }
- if (sb.length() > 0) {
- list.add(sb.toString());
- }
- return list;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 9a945f7..0000000
--- a/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.storm.sql.runtime.datasource.socket.SocketDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
deleted file mode 100644
index 174bfde..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
-import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestAvroSerializer {
- private static final String schemaString = "{\"type\":\"record\"," +
- "\"name\":\"avrotest\"," +
- "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
- "{ \"name\":\"val\", \"type\":\"string\" }]}";
-
- private static final String schemaString1 = "{\"type\":\"record\"," +
- "\"name\":\"avrotest\"," +
- "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
- "{ \"type\":{\"type\":\"map\",\"values\": \"long\"}, \"name\":\"val1\" }," +
- "{ \"type\":{\"type\":\"array\",\"items\": \"string\"}, \"name\":\"val2\" }]}";
-
- @Test
- public void testAvroSchemeAndSerializer() {
- List<String> fields = Lists.newArrayList("ID", "val");
- List<Object> o = Lists.newArrayList(1, "2");
-
- AvroSerializer serializer = new AvroSerializer(schemaString, fields);
- ByteBuffer byteBuffer = serializer.write(o, null);
-
- AvroScheme scheme = new AvroScheme(schemaString, fields);
- assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
- }
-
- @Test
- public void testAvroComplexSchemeAndSerializer() {
- List<String> fields = Lists.newArrayList("ID", "val1", "val2");
-
- Map<String,Long> mp = Maps.newHashMap();;
- mp.put("l1",1234L);
- mp.put("l2",56789L);
- List<String> ls = Lists.newArrayList("s1", "s2");
- List<Object> o = Lists.newArrayList(1, mp, ls);
-
- AvroSerializer serializer = new AvroSerializer(schemaString1, fields);
- ByteBuffer byteBuffer = serializer.write(o, null);
-
- AvroScheme scheme = new AvroScheme(schemaString1, fields);
- assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
deleted file mode 100644
index 0108949..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
-import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestCsvSerializer {
-
- @Test
- public void testCsvSchemeAndSerializer() {
- List<String> fields = Lists.newArrayList("ID", "val");
- List<Object> o = Lists.newArrayList("1", "2");
-
- CsvSerializer serializer = new CsvSerializer(fields);
- ByteBuffer byteBuffer = serializer.write(o, null);
-
- CsvScheme scheme = new CsvScheme(fields);
- assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-
- // Fields with embedded commas or double-quote characters
- fields = Lists.newArrayList("ID", "val", "v");
- o = Lists.newArrayList("1,9", "2,\"3\",5", "\"8\"");
-
- serializer = new CsvSerializer(fields);
- byteBuffer = serializer.write(o, null);
-
- scheme = new CsvScheme(fields);
- assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
deleted file mode 100644
index 6ca1877..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import org.apache.storm.sql.runtime.serde.json.JsonScheme;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.utils.Utils;
-import com.google.common.collect.Lists;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-public class TestJsonRepresentation {
- @Test
- public void testJsonScheme() {
- final List<String> fields = Lists.newArrayList("ID", "val");
- final String s = "{\"ID\": 1, \"val\": \"2\"}";
- JsonScheme scheme = new JsonScheme(fields);
- List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
- assertArrayEquals(new Object[] {1, "2"}, o.toArray());
- }
-
- @Test
- public void testJsonSerializer() {
- final List<String> fields = Lists.newArrayList("ID", "val");
- List<Object> o = Lists.<Object> newArrayList(1, "2");
- JsonSerializer s = new JsonSerializer(fields);
- ByteBuffer buf = s.write(o, null);
- byte[] b = Utils.toByteArray(buf);
- assertEquals("{\"ID\":1,\"val\":\"2\"}", new String(b));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
deleted file mode 100644
index 1798828..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
-import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestTsvSerializer {
-
- @Test
- public void testTsvSchemeAndSerializer() {
- final char delimiter = '\t';
-
- List<String> fields = Lists.newArrayList("ID", "val");
- List<Object> o = Lists.newArrayList("1", "2");
-
- TsvSerializer serializer = new TsvSerializer(fields, delimiter);
- ByteBuffer byteBuffer = serializer.write(o, null);
-
- TsvScheme scheme = new TsvScheme(fields, delimiter);
- assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
- }
-
-}
[04/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
new file mode 100644
index 0000000..82dc184
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
@@ -0,0 +1,492 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.tuple.Values;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class TestStormSql {
+ private static class MockDataSourceProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "mock";
+ }
+
+ @Override
+ public DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ return new TestUtils.MockDataSource();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ return new TestUtils.MockSqlTridentDataSource();
+ }
+ }
+
+ private static class MockNestedDataSourceProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "mocknested";
+ }
+
+ @Override
+ public DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ return new TestUtils.MockNestedDataSource();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ return new TestUtils.MockSqlTridentDataSource();
+ }
+ }
+
+ private static class MockGroupDataSourceProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "mockgroup";
+ }
+
+ @Override
+ public DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ return new TestUtils.MockGroupDataSource();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ return new TestUtils.MockSqlTridentGroupedDataSource();
+ }
+ }
+
+ private static class MockEmpDataSourceProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "mockemp";
+ }
+
+ @Override
+ public DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ return new TestUtils.MockEmpDataSource();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ return new TestUtils.MockSqlTridentJoinDataSourceEmp();
+ }
+ }
+
+ private static class MockDeptDataSourceProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "mockdept";
+ }
+
+ @Override
+ public DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ return new TestUtils.MockDeptDataSource();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ return new TestUtils.MockSqlTridentJoinDataSourceDept();
+ }
+ }
+
+
+ @BeforeClass
+ public static void setUp() {
+ 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());
+ }
+
+ @AfterClass
+ public static void tearDown() {
+ DataSourcesRegistry.providerMap().remove("mock");
+ DataSourcesRegistry.providerMap().remove("mocknested");
+ }
+
+ @Test
+ public void testExternalDataSource() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+ stmt.add("SELECT STREAM ID + 1 FROM FOO WHERE ID > 2");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(2, values.size());
+ Assert.assertEquals(4, values.get(0).get(0));
+ Assert.assertEquals(5, values.get(1).get(0));
+ }
+
+ @Test
+ public void testExternalDataSourceNested() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+ stmt.add("SELECT STREAM ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE CAST(MAPFIELD['b'] AS INTEGER) = 2 AND CAST(ARRAYFIELD[2] AS INTEGER) = 200");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ System.out.println(values);
+ Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+ Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+ Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
+ }
+
+ @Test
+ public void testExternalNestedNonExistKeyAccess() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ // this triggers java.lang.RuntimeException: Cannot convert null to int
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+ stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE CAST(MAPFIELD['a'] AS INTEGER) = 2");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(0, values.size());
+ }
+
+ @Test
+ public void testExternalNestedNonExistKeyAccess2() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ // this triggers java.lang.RuntimeException: Cannot convert null to int
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+ stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE CAST(NESTEDMAPFIELD['b']['c'] AS INTEGER) = 4");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(0, values.size());
+ }
+
+ @Test
+ public void testExternalNestedInvalidAccessStringIndexOnArray() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+ stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE CAST(ARRAYFIELD['a'] AS INTEGER) = 200");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(0, values.size());
+ }
+
+ @Test
+ public void testExternalNestedArrayOutOfBoundAccess() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+ stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE CAST(ARRAYFIELD[10] AS INTEGER) = 200");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(0, values.size());
+ }
+
+ @Test(expected = ValidationException.class)
+ public void testExternalUdfType() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
+ stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+ stmt.add("SELECT STREAM MYPLUS(NAME, 1) FROM FOO WHERE ID = 0");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ System.out.println(values);
+
+ }
+
+ @Test(expected = CompilingClassLoader.CompilerException.class)
+ public void testExternalUdfType2() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ // generated code will be not compilable since return type of MYPLUS and type of 'x' are different
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
+ stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+ stmt.add("SELECT STREAM ID FROM FOO WHERE MYPLUS(ID, 1) = 'x'");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(0, values.size());
+ }
+
+ @Test
+ public void testExternalUdf() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+ stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+ stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(2, values.size());
+ Assert.assertEquals(4, values.get(0).get(0));
+ Assert.assertEquals(5, values.get(1).get(0));
+ }
+
+ @Test(expected = UnsupportedOperationException.class)
+ public void testExternalUdfUsingJar() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+ stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus' USING JAR 'foo'");
+ stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ }
+
+ @Test
+ public void testGroupbyBuiltin() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(SALARY) FROM FOO GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(4, values.size());
+ Assert.assertEquals(3, values.get(0).get(2));
+ Assert.assertEquals(12, values.get(1).get(2));
+ Assert.assertEquals(21, values.get(2).get(2));
+ Assert.assertEquals(9, values.get(3).get(2));
+ }
+
+ @Test
+ public void testGroupbyBuiltinWithFilter() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(1, values.get(0).get(0));
+ Assert.assertEquals(3L, values.get(0).get(1));
+ Assert.assertEquals(12, values.get(0).get(2));
+ Assert.assertEquals(2.5, values.get(0).get(3));
+ }
+
+ @Test
+ public void testGroupbyBuiltinAndUDF() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("CREATE FUNCTION MYCONCAT AS 'org.apache.storm.sql.TestUtils$MyConcat'");
+ stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
+ stmt.add("SELECT STREAM ID, SUM(SALARY), MYCONCAT(NAME), TOPN(2, SALARY) FROM FOO GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(4, values.size());
+ Assert.assertEquals(3, values.get(0).get(1));
+ Assert.assertEquals("xxx", values.get(0).get(2));
+ Assert.assertEquals(Arrays.asList(2, 1), values.get(0).get(3));
+ Assert.assertEquals(12, values.get(1).get(1));
+ Assert.assertEquals("xxx", values.get(1).get(2));
+ Assert.assertEquals(Arrays.asList(5, 4), values.get(1).get(3));
+ Assert.assertEquals(21, values.get(2).get(1));
+ Assert.assertEquals("xxx", values.get(2).get(2));
+ Assert.assertEquals(Arrays.asList(8, 7), values.get(2).get(3));
+ Assert.assertEquals(9, values.get(3).get(1));
+ Assert.assertEquals("x", values.get(3).get(2));
+ Assert.assertEquals(Arrays.asList(9), values.get(3).get(3));
+ }
+
+ @Test
+ public void testAggFnNonSqlReturnType() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
+ stmt.add("SELECT STREAM ID, SUM(SALARY), TOPN(1, SALARY) FROM FOO WHERE ID >= 0 GROUP BY (ID) HAVING MAX(SALARY) > 0");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(4, values.size());
+ Assert.assertEquals(Collections.singletonList(2), values.get(0).get(2));
+ Assert.assertEquals(Collections.singletonList(5), values.get(1).get(2));
+ Assert.assertEquals(Collections.singletonList(8), values.get(2).get(2));
+ Assert.assertEquals(Collections.singletonList(9), values.get(3).get(2));
+ }
+
+ @Test
+ public void testGroupbySameAggregateOnDifferentColumns() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, COUNT(*), AVG(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(1, values.get(0).get(0));
+ Assert.assertEquals(3L, values.get(0).get(1));
+ Assert.assertEquals(4, values.get(0).get(2));
+ Assert.assertEquals(2.5, values.get(0).get(3));
+ }
+
+ @Test(expected = UnsupportedOperationException.class)
+ public void testGroupbyBuiltinNotimplemented() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, COUNT(*), STDDEV_POP(SALARY) FROM FOO GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ }
+
+ @Test
+ public void testMinMax() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY (ID)");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(4, values.size());
+ Assert.assertEquals(0, values.get(0).get(2));
+ Assert.assertEquals(3, values.get(1).get(2));
+ Assert.assertEquals(6, values.get(2).get(2));
+ Assert.assertEquals(9, values.get(3).get(2));
+
+ Assert.assertEquals(1.5, values.get(0).get(3));
+ Assert.assertEquals(3.0, values.get(1).get(3));
+ Assert.assertEquals(4.5, values.get(2).get(3));
+ Assert.assertEquals(5.0, values.get(3).get(3));
+ }
+ @Test
+ public void testFilterGroupbyHaving() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM ID, MIN(SALARY) FROM FOO where ID > 0 GROUP BY (ID) HAVING ID > 2 AND MAX(SALARY) > 5");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(3, values.get(0).get(0));
+ Assert.assertEquals(9, values.get(0).get(1));
+ }
+
+ @Test
+ public void testGroupByMultipleFields() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE FOO (DEPTID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR, EMPID INT) LOCATION 'mockgroup:///foo'");
+ stmt.add("SELECT STREAM DEPTID, EMPID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY DEPTID, EMPID");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(7, values.size());
+ Assert.assertEquals(0, values.get(0).get(0));
+ Assert.assertEquals(0, values.get(0).get(1));
+ Assert.assertEquals(2L, values.get(0).get(2));
+ }
+
+ @Test
+ public void testjoin() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
+ stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
+ stmt.add("SELECT STREAM EMPID, EMPNAME, DEPTNAME FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ System.out.println(values);
+ Assert.assertEquals(3, values.size());
+ Assert.assertEquals("emp1", values.get(0).get(1));
+ Assert.assertEquals("dept1", values.get(0).get(2));
+ Assert.assertEquals("emp2", values.get(1).get(1));
+ Assert.assertEquals("dept1", values.get(1).get(2));
+ Assert.assertEquals("emp3", values.get(2).get(1));
+ Assert.assertEquals("dept2", values.get(2).get(2));
+ }
+
+ @Test
+ public void testjoinAndGroupby() throws Exception {
+ List<String> stmt = new ArrayList<>();
+ stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
+ stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
+ stmt.add("SELECT STREAM d.DEPTID, count(EMPID) FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0" +
+ "GROUP BY d.DEPTID");
+ StormSql sql = StormSql.construct();
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ sql.execute(stmt, h);
+ Assert.assertEquals(2, values.size());
+ Assert.assertEquals(1, values.get(0).get(0));
+ Assert.assertEquals(2L, values.get(0).get(1));
+ Assert.assertEquals(2, values.get(1).get(0));
+ Assert.assertEquals(1L, values.get(1).get(1));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
new file mode 100644
index 0000000..634e454
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
@@ -0,0 +1,410 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler;
+
+import com.google.common.base.Function;
+import org.apache.storm.sql.compiler.backends.standalone.TestCompilerUtils;
+import org.apache.storm.tuple.Values;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestExprSemantic {
+ private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+ RelDataTypeSystem.DEFAULT);
+
+ @Test
+ public void testLogicalExpr() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList("ID > 0 OR ID < 1", "ID > 0 AND ID < 1",
+ "NOT (ID > 0 AND ID < 1)"));
+ assertEquals(new Values(true, false, true), v);
+ }
+
+ @Test
+ public void testExpectOperator() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList("TRUE IS TRUE", "TRUE IS NOT TRUE",
+ "UNKNOWN IS TRUE", "UNKNOWN IS NOT TRUE",
+ "TRUE IS FALSE", "UNKNOWN IS NULL",
+ "UNKNOWN IS NOT NULL"));
+ assertEquals(new Values(true, false, false, true, false, true, false), v);
+ }
+
+ @Test
+ public void testDistinctBetweenLikeSimilarIn() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList("TRUE IS DISTINCT FROM TRUE",
+ "TRUE IS NOT DISTINCT FROM FALSE", "3 BETWEEN 1 AND 5",
+ "10 NOT BETWEEN 1 AND 5", "'hello' LIKE '_e%'",
+ "'world' NOT LIKE 'wor%'", "'abc' SIMILAR TO '[a-zA-Z]+[cd]{1}'",
+ "'abe' NOT SIMILAR TO '[a-zA-Z]+[cd]{1}'", "'3' IN ('1', '2', '3', '4')",
+ "2 NOT IN (1, 3, 5)"));
+ assertEquals(new Values(false, false, true, true, true,
+ false, true, true, true, true), v);
+ }
+
+ @Test
+ public void testCaseStatement() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "CASE WHEN 'abcd' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+ "WHEN UPPER('abcd') = 'AB' THEN 'b' ELSE {fn CONCAT('abcd', '#')} END",
+ "CASE WHEN 'ab' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+ "WHEN UPPER('ab') = 'AB' THEN 'b' ELSE {fn CONCAT('ab', '#')} END",
+ "CASE WHEN 'abc' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+ "WHEN UPPER('abc') = 'AB' THEN 'b' ELSE {fn CONCAT('abc', '#')} END"
+ )
+ );
+
+ // TODO: The data type of literal Calcite assigns seems to be out of expectation. Please see below logical plan.
+ // LogicalProject(EXPR$0=[CASE(OR(=('abcd', 'a'), =('abcd', 'abc'), =('abcd', 'abcde')), CAST(UPPER('a')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abcd'), CAST('AB'):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), 'b', CAST(||('abcd', '#')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL)], EXPR$1=[CASE(OR(=('ab', 'a'), =('ab', 'abc'), =('ab', 'abcde')), CAST(UPPER('a')):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('ab'), 'AB'), CAST('b'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('ab', '#'))], EXPR$2=[CASE(OR(=('abc', 'a'), =('abc', 'abc'), =('abc', 'abcde')), CAST(UPPER('a')):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abc'), CAST('AB'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), CAST('b'):CHAR(4) C
HARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('abc', '#'))]): rowcount = 1.0, cumulative cost = {2.0 rows, 5.0 cpu, 0.0 io}, id = 5
+ // LogicalFilter(condition=[AND(>($0, 0), <($0, 2))]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 4
+ // EnumerableTableScan(table=[[FOO]]): rowcount = 1.0, cumulative cost = {0.0 rows, 1.0 cpu, 0.0 io}, id = 3
+ // in result, both 'b' and UPPER('a') hence 'A' are having some spaces which is not expected.
+ // When we use CASE with actual column (Java String type hence VARCHAR), it seems to work as expected.
+ // Please refer trident/TestPlanCompiler#testCaseStatement(), and see below logical plan.
+ // LogicalProject(EXPR$0=[CASE(OR(=($1, 'a'), =($1, 'abc'), =($1, 'abcde')), CAST(UPPER('a')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", =(CAST(UPPER($1)):VARCHAR(2) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", 'AB'), 'b', CAST(||($1, '#')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary")]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 3
+ List<Object> v2 = Lists.transform(v, new Function<Object, Object>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable Object o) {
+ return ((String) o).trim();
+ }
+ });
+ assertArrayEquals(new Values("abcd#", "b", "A").toArray(), v2.toArray());
+ }
+
+ @Test
+ public void testNullIfAndCoalesce() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "NULLIF(5, 5)", "NULLIF(5, 0)", "COALESCE(NULL, NULL, 5, 4, NULL)", "COALESCE(1, 5)"
+ ));
+ assertEquals(new Values(null, 5, 5, 1), v);
+ }
+
+ @Test
+ public void testCollectionFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "ELEMENT(ARRAY[3])", "CARDINALITY(ARRAY[1, 2, 3, 4, 5])"
+ ));
+ assertEquals(new Values(3, 5), v);
+ }
+
+ @Test(expected = RuntimeException.class)
+ public void testElementFunctionMoreThanOneValue() throws Exception {
+ testExpr(
+ Lists.newArrayList(
+ "ELEMENT(ARRAY[1, 2, 3])"
+ ));
+ fail("ELEMENT with array which has multiple elements should throw exception in runtime.");
+ }
+
+ @Test
+ public void testArithmeticWithNull() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "1 + CAST(NULL AS INT)", "CAST(NULL AS INT) + 1", "CAST(NULL AS INT) + CAST(NULL AS INT)", "1 + 2"
+ ));
+ assertEquals(new Values(null, null, null, 3), v);
+ }
+
+ @Test
+ public void testNotWithNull() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "NOT TRUE", "NOT FALSE", "NOT UNKNOWN"
+ ));
+ assertEquals(new Values(false, true, null), v);
+ }
+
+ @Test
+ public void testAndWithNull() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "UNKNOWN AND TRUE", "UNKNOWN AND FALSE", "UNKNOWN AND UNKNOWN",
+ "TRUE AND TRUE", "TRUE AND FALSE", "TRUE AND UNKNOWN",
+ "FALSE AND TRUE", "FALSE AND FALSE", "FALSE AND UNKNOWN"
+ ));
+ assertEquals(new Values(null, false, null, true, false, null, false,
+ false, false), v);
+ }
+
+ @Test
+ public void testAndWithNullable() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "ADDR = 'a' AND NAME = 'a'", "NAME = 'a' AND ADDR = 'a'", "NAME = 'x' AND ADDR = 'a'", "ADDR = 'a' AND NAME = 'x'"
+ ));
+ assertEquals(new Values(false, false, null, null), v);
+ }
+
+ @Test
+ public void testOrWithNullable() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "ADDR = 'a' OR NAME = 'a'", "NAME = 'a' OR ADDR = 'a' ", "NAME = 'x' OR ADDR = 'a' ", "ADDR = 'a' OR NAME = 'x'"
+ ));
+ assertEquals(new Values(null, null, true, true), v);
+ }
+
+ @Test
+ public void testOrWithNull() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "UNKNOWN OR TRUE", "UNKNOWN OR FALSE", "UNKNOWN OR UNKNOWN",
+ "TRUE OR TRUE", "TRUE OR FALSE", "TRUE OR UNKNOWN",
+ "FALSE OR TRUE", "FALSE OR FALSE", "FALSE OR UNKNOWN"
+ ));
+ assertEquals(new Values(true, null, null, true, true, true, true,
+ false, null), v);
+ }
+
+ @Test
+ public void testEquals() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "1 = 2", "UNKNOWN = UNKNOWN", "'a' = 'a'", "'a' = UNKNOWN", "UNKNOWN = 'a'", "'a' = 'b'",
+ "1 <> 2", "UNKNOWN <> UNKNOWN", "'a' <> 'a'", "'a' <> UNKNOWN", "UNKNOWN <> 'a'", "'a' <> 'b'"
+ ));
+ assertEquals(new Values(false, null, true, null, null, false,
+ true, null, false, null, null, true), v);
+ }
+
+ @Test
+ public void testArithmeticFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "POWER(3, 2)", "ABS(-10)", "MOD(10, 3)", "MOD(-10, 3)",
+ "CEIL(123.45)", "FLOOR(123.45)"
+ ));
+
+ assertEquals(new Values(9.0d, 10, 1, -1, new BigDecimal(124), new BigDecimal(123)), v);
+
+ // Belows are floating numbers so comparing this with literal is tend to be failing...
+ // Picking int value and compare
+ Values v2 = testExpr(
+ Lists.newArrayList(
+ "SQRT(255)", "LN(16)", "LOG10(10000)", "EXP(10)"
+ ));
+ List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
+ @Nullable
+ @Override
+ public Object apply(@Nullable Object o) {
+ // only takes int value
+ return ((Number) o).intValue();
+ }
+ });
+
+ // 15.9687, 2.7725, 4.0, 22026.465794
+ assertEquals(new Values(15, 2, 4, 22026), v2m);
+ }
+
+ @Test
+ public void testStringFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "'ab' || 'cd'", "CHAR_LENGTH('foo')", "CHARACTER_LENGTH('foo')",
+ "UPPER('a')", "LOWER('A')", "POSITION('bc' IN 'abcd')",
+ "TRIM(BOTH ' ' FROM ' abcdeabcdeabc ')",
+ "TRIM(LEADING ' ' FROM ' abcdeabcdeabc ')",
+ "TRIM(TRAILING ' ' FROM ' abcdeabcdeabc ')",
+ "OVERLAY('abcde' PLACING 'bc' FROM 3)",
+ "SUBSTRING('abcde' FROM 3)", "SUBSTRING('abcdeabcde' FROM 3 FOR 4)",
+ "INITCAP('foo')"
+ ));
+ assertEquals(new Values("abcd", 3, 3, "A", "a", 2, "abcdeabcdeabc", "abcdeabcdeabc ", " abcdeabcdeabc", "abbce", "cde", "cdea", "Foo"), v);
+ }
+
+ @Test
+ public void testBinaryStringFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "x'45F0AB' || x'45F0AB'",
+ "POSITION(x'F0' IN x'453423F0ABBC')",
+ "OVERLAY(x'453423F0ABBC45' PLACING x'4534' FROM 3)"
+ // "SUBSTRING(x'453423F0ABBC' FROM 3)",
+ // "SUBSTRING(x'453423F0ABBC453423F0ABBC' FROM 3 FOR 4)"
+ ));
+
+ // TODO: Calcite 1.9.0 has bugs on binary SUBSTRING functions
+ // as there's no SqlFunctions.substring(org.apache.calcite.avatica.util.ByteString, ...)
+ // commented out testing substring function
+
+ assertEquals("45f0ab45f0ab", v.get(0).toString());
+ assertEquals(4, v.get(1));
+ assertEquals("45344534abbc45", v.get(2).toString());
+ // assertEquals("23f0abbc", v.get(3).toString());
+ // assertEquals("23f0ab", v.get(4).toString());
+ }
+
+ @Test
+ public void testDateAndTimestampLiteral() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "DATE '1970-05-15' AS datefield",
+ "TIME '00:00:00' AS timefield",
+ "TIMESTAMP '2016-01-01 00:00:00' as timestampfield"
+ )
+ );
+
+ assertEquals(3, v.size());
+ assertEquals(134, v.get(0));
+ assertEquals(0, v.get(1));
+ assertEquals(1451606400000L, v.get(2));
+ }
+
+ @Test
+ public void testInterval() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "INTERVAL '1-5' YEAR TO MONTH AS intervalfield",
+ "(DATE '1970-01-01', DATE '1970-01-15') AS anchoredinterval_field"
+ )
+ );
+
+ assertEquals(3, v.size());
+ assertEquals(17, v.get(0));
+ assertEquals(0, v.get(1));
+ assertEquals(14, v.get(2));
+ }
+
+ @Test
+ public void testDateFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "LOCALTIME = CURRENT_TIME, LOCALTIMESTAMP = CURRENT_TIMESTAMP, CURRENT_DATE",
+ "EXTRACT(MONTH FROM TIMESTAMP '2010-01-23 12:34:56')",
+ "FLOOR(DATE '2016-01-23' TO MONTH)",
+ "CEIL(TIME '12:34:56' TO MINUTE)"
+ )
+ );
+
+ assertEquals(6, v.size());
+ assertTrue((boolean) v.get(0));
+ assertTrue((boolean) v.get(1));
+ // skip checking CURRENT_DATE since we don't inject dataContext so don't know about current timestamp
+ // we can do it from trident test
+ assertEquals(1L, v.get(3));
+ assertEquals(0L, v.get(4));
+ assertEquals(45300000, v.get(5));
+ }
+
+ @Test
+ public void testJDBCNumericFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "{fn POWER(3, 2)}", "{fn ABS(-10)}", "{fn MOD(10, 3)}", "{fn MOD(-10, 3)}"
+ ));
+
+ assertEquals(new Values(9.0d, 10, 1, -1), v);
+
+ // Belows are floating numbers so comparing this with literal is tend to be failing...
+ // Picking int value and compare
+ Values v2 = testExpr(
+ Lists.newArrayList(
+ "{fn LOG(16)}", "{fn LOG10(10000)}", "{fn EXP(10)}"
+ ));
+ List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
+ @Nullable
+ @Override
+ public Object apply(@Nullable Object o) {
+ // only takes int value
+ return ((Number) o).intValue();
+ }
+ });
+
+ // 2.7725, 4.0, 22026.465794
+ assertEquals(new Values(2, 4, 22026), v2m);
+ }
+
+ @Test
+ public void testJDBCStringFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "{fn CONCAT('ab', 'cd')}",
+ "{fn LOCATE('bc', 'abcdeabcde')}",
+ //"{fn LOCATE('bc', 'abcdeabcde', 4)}",
+ "{fn INSERT('abcd', 2, 3, 'de')}",
+ "{fn LCASE('AbCdE')}",
+ "{fn LENGTH('AbCdE')}",
+ //"{fn LTRIM(' abcde ')}",
+ //"{fn RTRIM(' abcde ')}",
+ "{fn SUBSTRING('abcdeabcde', 3, 4)}",
+ "{fn UCASE('AbCdE')}"
+ )
+ );
+
+ // TODO: Calcite 1.9.0 doesn't support {fn LOCATE(string1, string2 [, integer])}
+ // while it's on support list on SQL reference
+ // and bugs on LTRIM and RTRIM : throwing AssertionError: Internal error: pre-condition failed: pos != null
+ // commented out problematic function tests
+
+ assertEquals(new Values("abcd", 2, "ade", "abcde", 5, "cdea", "ABCDE"), v);
+ }
+
+ @Test
+ public void testJDBCDateTimeFunctions() throws Exception {
+ Values v = testExpr(
+ Lists.newArrayList(
+ "{fn CURDATE()} = CURRENT_DATE", "{fn CURTIME()} = LOCALTIME", "{fn NOW()} = LOCALTIMESTAMP",
+ "{fn QUARTER(DATE '2016-10-07')}", "{fn TIMESTAMPADD(MINUTE, 15, TIMESTAMP '2016-10-07 00:00:00')}",
+ "{fn TIMESTAMPDIFF(SECOND, TIMESTAMP '2016-10-06 00:00:00', TIMESTAMP '2016-10-07 00:00:00')}"
+ )
+ );
+
+ assertEquals(new Values(true, true, true, 4L, 1475799300000L, 86400), v);
+ }
+
+ private Values testExpr(List<String> exprs) throws Exception {
+ String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
+ " WHERE ID > 0 AND ID < 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(state.tree());
+ Map<String, DataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockDataSource());
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ proc.initialize(data, h);
+ return values.get(0);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
new file mode 100644
index 0000000..8e64e9c
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.StreamableTable;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.compiler.CompilerUtil;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class TestCompilerUtils {
+
+ public static class MyPlus {
+ public static Integer eval(Integer x, Integer y) {
+ return x + y;
+ }
+ }
+
+ public static class MyStaticSumFunction {
+ public static long init() {
+ return 0L;
+ }
+ public static long add(long accumulator, int v) {
+ return accumulator + v;
+ }
+ }
+
+ public static class MySumFunction {
+ public MySumFunction() {
+ }
+ public long init() {
+ return 0L;
+ }
+ public long add(long accumulator, int v) {
+ return accumulator + v;
+ }
+ public long result(long accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static CalciteState sqlOverDummyTable(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("ID", SqlTypeName.INTEGER)
+ .field("NAME", typeFactory.createType(String.class))
+ .field("ADDR", typeFactory.createType(String.class))
+ .build();
+ Table table = streamableTable.stream();
+ schema.add("FOO", table);
+ schema.add("BAR", table);
+ schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+ List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+ sqlOperatorTables.add(SqlStdOperatorTable.instance());
+ sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+ false,
+ Collections.<String>emptyList(), typeFactory));
+ SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
+ FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+ schema).operatorTable(chainedSqlOperatorTable).build();
+ Planner planner = Frameworks.getPlanner(config);
+ SqlNode parse = planner.parse(sql);
+ SqlNode validate = planner.validate(parse);
+ RelNode tree = planner.convert(validate);
+ System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static CalciteState sqlOverNestedTable(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("ID", SqlTypeName.INTEGER)
+ .field("MAPFIELD",
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+ , true))
+ .field("NESTEDMAPFIELD",
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+ , true))
+ , true))
+ .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
+ typeFactory.createArrayType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
+ , true))
+ .build();
+ Table table = streamableTable.stream();
+ schema.add("FOO", table);
+ schema.add("BAR", table);
+ schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+ List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+ sqlOperatorTables.add(SqlStdOperatorTable.instance());
+ sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+ false,
+ Collections.<String>emptyList(), typeFactory));
+ SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
+ FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+ schema).operatorTable(chainedSqlOperatorTable).build();
+ Planner planner = Frameworks.getPlanner(config);
+ SqlNode parse = planner.parse(sql);
+ SqlNode validate = planner.validate(parse);
+ RelNode tree = planner.convert(validate);
+ System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static class CalciteState {
+ final SchemaPlus schema;
+ final RelNode tree;
+
+ private CalciteState(SchemaPlus schema, RelNode tree) {
+ this.schema = schema;
+ this.tree = tree;
+ }
+
+ public SchemaPlus schema() {
+ return schema;
+ }
+
+ public RelNode tree() {
+ return tree;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
new file mode 100644
index 0000000..3226810
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.storm.tuple.Values;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestPlanCompiler {
+ private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+ RelDataTypeSystem.DEFAULT);
+
+ @Test
+ public void testCompile() throws Exception {
+ String sql = "SELECT ID + 1 FROM FOO WHERE ID > 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(state.tree());
+ Map<String, DataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockDataSource());
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ proc.initialize(data, h);
+ Assert.assertArrayEquals(new Values[] { new Values(4), new Values(5)},
+ values.toArray());
+ }
+
+ @Test
+ public void testLogicalExpr() throws Exception {
+ String sql = "SELECT ID > 0 OR ID < 1, ID > 0 AND ID < 1, NOT (ID > 0 AND ID < 1) FROM FOO WHERE ID > 0 AND ID < 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(state.tree());
+ Map<String, DataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockDataSource());
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ proc.initialize(data, h);
+ Assert.assertEquals(new Values(true, false, true), values.get(0));
+ }
+
+ @Test
+ public void testNested() throws Exception {
+ String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(state.tree());
+ Map<String, DataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockNestedDataSource());
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ proc.initialize(data, h);
+ Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+ Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+ Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
+ }
+
+ @Test
+ public void testUdf() throws Exception {
+ String sql = "SELECT MYPLUS(ID, 3)" +
+ "FROM FOO " +
+ "WHERE ID = 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(state.tree());
+ Map<String, DataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockDataSource());
+ List<Values> values = new ArrayList<>();
+ ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+ proc.initialize(data, h);
+ Assert.assertEquals(new Values(5), values.get(0));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
new file mode 100644
index 0000000..4bee9aa
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.standalone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+public class TestRelNodeCompiler {
+ @Test
+ public void testFilter() throws Exception {
+ String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+ RelDataTypeSystem.DEFAULT);
+ LogicalProject project = (LogicalProject) state.tree();
+ LogicalFilter filter = (LogicalFilter) project.getInput();
+
+ try (StringWriter sw = new StringWriter();
+ PrintWriter pw = new PrintWriter(sw)
+ ) {
+ RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+ // standalone mode doesn't use inputstreams argument
+ compiler.visitFilter(filter, Collections.EMPTY_LIST);
+ pw.flush();
+ Assert.assertThat(sw.toString(), containsString("> 3"));
+ }
+
+ try (StringWriter sw = new StringWriter();
+ PrintWriter pw = new PrintWriter(sw)
+ ) {
+ RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+ // standalone mode doesn't use inputstreams argument
+ compiler.visitProject(project, Collections.EMPTY_LIST);
+ pw.flush();
+ Assert.assertThat(sw.toString(), containsString(" + 1"));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
new file mode 100644
index 0000000..f6ef1ca
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.compiler.backends.trident;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.StreamableTable;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.compiler.CompilerUtil;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.apache.storm.sql.planner.StormRelUtils;
+
+public class TestCompilerUtils {
+
+ public static class MyPlus {
+ public static Integer eval(Integer x, Integer y) {
+ return x + y;
+ }
+ }
+
+ public static class MyStaticSumFunction {
+ public static long init() {
+ return 0L;
+ }
+ public static long add(long accumulator, int v) {
+ return accumulator + v;
+ }
+ }
+
+ public static class MySumFunction {
+ public MySumFunction() {
+ }
+ public long init() {
+ return 0L;
+ }
+ public long add(long accumulator, int v) {
+ return accumulator + v;
+ }
+ public long result(long accumulator) {
+ return accumulator;
+ }
+ }
+
+ public static CalciteState sqlOverDummyTable(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("ID", SqlTypeName.INTEGER)
+ .field("NAME", typeFactory.createType(String.class))
+ .field("ADDR", typeFactory.createType(String.class))
+ .build();
+ Table table = streamableTable.stream();
+ schema.add("FOO", table);
+ schema.add("BAR", table);
+ schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+ QueryPlanner queryPlanner = new QueryPlanner(schema);
+ TridentRel tree = queryPlanner.getPlan(sql);
+ System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static CalciteState sqlOverDummyGroupByTable(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("ID", SqlTypeName.INTEGER)
+ .field("GRPID", SqlTypeName.INTEGER)
+ .field("NAME", typeFactory.createType(String.class))
+ .field("ADDR", typeFactory.createType(String.class))
+ .field("AGE", SqlTypeName.INTEGER)
+ .field("SCORE", SqlTypeName.INTEGER)
+ .build();
+ Table table = streamableTable.stream();
+ schema.add("FOO", table);
+ schema.add("BAR", table);
+ schema.add("MYSTATICSUM", AggregateFunctionImpl.create(MyStaticSumFunction.class));
+ schema.add("MYSUM", AggregateFunctionImpl.create(MySumFunction.class));
+
+ QueryPlanner queryPlanner = new QueryPlanner(schema);
+ TridentRel tree = queryPlanner.getPlan(sql);
+ System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static CalciteState sqlOverNestedTable(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("ID", SqlTypeName.INTEGER)
+ .field("MAPFIELD",
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+ , true))
+ .field("NESTEDMAPFIELD",
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createMapType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+ , true))
+ , true))
+ .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
+ typeFactory.createArrayType(
+ typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
+ , true))
+ .build();
+ Table table = streamableTable.stream();
+ schema.add("FOO", table);
+ schema.add("BAR", table);
+ schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+ QueryPlanner queryPlanner = new QueryPlanner(schema);
+ TridentRel tree = queryPlanner.getPlan(sql);
+ System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static CalciteState sqlOverSimpleEquiJoinTables(String sql)
+ throws RelConversionException, ValidationException, SqlParseException {
+ SchemaPlus schema = Frameworks.createRootSchema(true);
+ JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+ (RelDataTypeSystem.DEFAULT);
+
+ StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("EMPID", SqlTypeName.INTEGER)
+ .field("EMPNAME", SqlTypeName.VARCHAR)
+ .field("DEPTID", SqlTypeName.INTEGER)
+ .build();
+ Table table = streamableTable.stream();
+
+ StreamableTable streamableTable2 = new CompilerUtil.TableBuilderInfo(typeFactory)
+ .field("DEPTID", SqlTypeName.INTEGER)
+ .field("DEPTNAME", SqlTypeName.VARCHAR)
+ .build();
+ Table table2 = streamableTable2.stream();
+
+ schema.add("EMP", table);
+ schema.add("DEPT", table2);
+
+ QueryPlanner queryPlanner = new QueryPlanner(schema);
+ TridentRel tree = queryPlanner.getPlan(sql);
+ System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+ return new CalciteState(schema, tree);
+ }
+
+ public static class CalciteState {
+ final SchemaPlus schema;
+ final RelNode tree;
+
+ private CalciteState(SchemaPlus schema, RelNode tree) {
+ this.schema = schema;
+ this.tree = tree;
+ }
+
+ public SchemaPlus schema() {
+ return schema;
+ }
+
+ public RelNode tree() {
+ return tree;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
new file mode 100644
index 0000000..d3f30b9
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
@@ -0,0 +1,232 @@
+/*
+ * *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements. See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership. The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License. You may obtain a copy of the License at
+ * * <p>
+ * * http://www.apache.org/licenses/LICENSE-2.0
+ * * <p>
+ * * Unless required by applicable law or agreed to in writing, software
+ * * distributed under the License is distributed on an "AS IS" BASIS,
+ * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * * See the License for the specific language governing permissions and
+ * * limitations under the License.
+ *
+ */
+package org.apache.storm.sql.compiler.backends.trident;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.AbstractTridentProcessor;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.time.ZoneOffset;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import static org.apache.storm.sql.TestUtils.MockState.getCollectedValues;
+
+public class TestPlanCompiler {
+ private static LocalCluster cluster;
+
+ @BeforeClass
+ public static void staticSetup() throws Exception {
+ cluster = new LocalCluster();
+ }
+
+ @AfterClass
+ public static void staticCleanup() {
+ if (cluster!= null) {
+ cluster.shutdown();
+ cluster = null;
+ }
+ }
+
+ @Before
+ public void setUp() {
+ getCollectedValues().clear();
+ }
+
+ @Test
+ public void testCompile() throws Exception {
+ final int EXPECTED_VALUE_SIZE = 2;
+ String sql = "SELECT ID FROM FOO WHERE ID > 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final TridentTopology topo = proc.build();
+ Fields f = proc.outputStream().getOutputFields();
+ proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
+ f, new TestUtils.MockStateUpdater(), new Fields());
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+ Assert.assertArrayEquals(new Values[] { new Values(3), new Values(4)}, getCollectedValues().toArray());
+ }
+
+ @Test
+ public void testInsert() throws Exception {
+ final int EXPECTED_VALUE_SIZE = 1;
+ String sql = "INSERT INTO BAR SELECT ID, NAME, ADDR FROM FOO WHERE ID > 3";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+ data.put("BAR", new TestUtils.MockSqlTridentDataSource());
+
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final TridentTopology topo = proc.build();
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+ Assert.assertArrayEquals(new Values[] { new Values(4, "abcde", "y")}, getCollectedValues().toArray());
+ }
+
+ @Test
+ public void testUdf() throws Exception {
+ int EXPECTED_VALUE_SIZE = 1;
+ String sql = "SELECT MYPLUS(ID, 3)" +
+ "FROM FOO " +
+ "WHERE ID = 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+ Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final TridentTopology topo = proc.build();
+ Fields f = proc.outputStream().getOutputFields();
+ proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
+ f, new TestUtils.MockStateUpdater(), new Fields());
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+ Assert.assertArrayEquals(new Values[] { new Values(5) }, getCollectedValues().toArray());
+ }
+
+ @Test
+ public void testCaseStatement() throws Exception {
+ int EXPECTED_VALUE_SIZE = 5;
+ String sql = "SELECT CASE WHEN NAME IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+ "WHEN UPPER(NAME) = 'AB' THEN 'b' ELSE {fn CONCAT(NAME, '#')} END FROM FOO";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+
+ final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final TridentTopology topo = proc.build();
+ Fields f = proc.outputStream().getOutputFields();
+ proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+ Assert.assertArrayEquals(new Values[]{new Values("A"), new Values("b"), new Values("A"), new Values("abcd#"), new Values("A")}, getCollectedValues().toArray());
+ }
+
+ @Test
+ public void testNested() throws Exception {
+ int EXPECTED_VALUE_SIZE = 1;
+ String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+ "FROM FOO " +
+ "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+
+ final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentNestedDataSource());
+
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final TridentTopology topo = proc.build();
+ Fields f = proc.outputStream().getOutputFields();
+ proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+ Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+ Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+ Assert.assertArrayEquals(new Values[]{new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300))}, getCollectedValues().toArray());
+ }
+
+ @Test
+ public void testDateKeywords() throws Exception {
+ int EXPECTED_VALUE_SIZE = 1;
+ String sql = "SELECT " +
+ "LOCALTIME, CURRENT_TIME, LOCALTIMESTAMP, CURRENT_TIMESTAMP, CURRENT_DATE " +
+ "FROM FOO " +
+ "WHERE ID > 0 AND ID < 2";
+ TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+
+ final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+ data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+ QueryPlanner planner = new QueryPlanner(state.schema());
+ AbstractTridentProcessor proc = planner.compile(data, sql);
+ final DataContext dataContext = proc.getDataContext();
+ final TridentTopology topo = proc.build();
+ Fields f = proc.outputStream().getOutputFields();
+ proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+ runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+ long utcTimestamp = (long) dataContext.get(DataContext.Variable.UTC_TIMESTAMP.camelName);
+ long currentTimestamp = (long) dataContext.get(DataContext.Variable.CURRENT_TIMESTAMP.camelName);
+ long localTimestamp = (long) dataContext.get(DataContext.Variable.LOCAL_TIMESTAMP.camelName);
+
+ System.out.println(getCollectedValues());
+
+ java.sql.Timestamp timestamp = new java.sql.Timestamp(utcTimestamp);
+ int dateInt = (int) timestamp.toLocalDateTime().atOffset(ZoneOffset.UTC).toLocalDate().toEpochDay();
+ int localTimeInt = (int) (localTimestamp % DateTimeUtils.MILLIS_PER_DAY);
+ int currentTimeInt = (int) (currentTimestamp % DateTimeUtils.MILLIS_PER_DAY);
+
+ Assert.assertArrayEquals(new Values[]{new Values(localTimeInt, currentTimeInt, localTimestamp, currentTimestamp, dateInt)}, getCollectedValues().toArray());
+ }
+
+ private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc,
+ TridentTopology topo) throws Exception {
+ final Config conf = new Config();
+ conf.setMaxSpoutPending(20);
+
+ if (proc.getClassLoaders() != null && proc.getClassLoaders().size() > 0) {
+ CompilingClassLoader lastClassloader = proc.getClassLoaders().get(proc.getClassLoaders().size() - 1);
+ Utils.setClassLoaderForJavaDeSerialize(lastClassloader);
+ }
+
+ try (LocalTopology stormTopo = cluster.submitTopology("storm-sql", conf, topo.build())) {
+ waitForCompletion(1000 * 1000, new Callable<Boolean>() {
+ @Override
+ public Boolean call() throws Exception {
+ return getCollectedValues().size() < expectedValueSize;
+ }
+ });
+ } finally {
+ while(cluster.getClusterInfo().get_topologies_size() > 0) {
+ Thread.sleep(10);
+ }
+ Utils.resetClassLoaderForJavaDeSerialize();
+ }
+ }
+
+ private void waitForCompletion(long timeout, Callable<Boolean> cond) throws Exception {
+ long start = TestUtils.monotonicNow();
+ while (TestUtils.monotonicNow() - start < timeout && cond.call()) {
+ Thread.sleep(100);
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
new file mode 100644
index 0000000..68054d8
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.parser;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.storm.sql.parser.impl.ParseException;
+import org.junit.Test;
+
+public class TestSqlParser {
+ @Test
+ public void testCreateTable() throws Exception {
+ String sql = "CREATE EXTERNAL TABLE foo (bar INT) LOCATION 'kafka:///foo'";
+ parse(sql);
+ }
+
+ @Test
+ public void testCreateTableWithPrimaryKey() throws Exception {
+ String sql = "CREATE EXTERNAL TABLE foo (bar INT PRIMARY KEY ASC) LOCATION 'kafka:///foo'";
+ parse(sql);
+ }
+
+ @Test(expected = ParseException.class)
+ public void testCreateTableWithoutLocation() throws Exception {
+ String sql = "CREATE EXTERNAL TABLE foo (bar INT)";
+ parse(sql);
+ }
+
+ @Test
+ public void testCreateFunction() throws Exception {
+ String sql = "CREATE FUNCTION foo AS 'org.apache.storm.sql.MyUDF'";
+ parse(sql);
+ }
+
+ private static SqlNode parse(String sql) throws Exception {
+ StormParser parser = new StormParser(sql);
+ return parser.impl().parseSqlStmtEof();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/pom.xml b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
new file mode 100644
index 0000000..9275fbb
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
@@ -0,0 +1,104 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-hdfs</artifactId>
+
+ <developers>
+ <developer>
+ <id>vesense</id>
+ <name>Xin Wang</name>
+ <email>data.xinwang@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ <exclusions>
+ <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+ see: http://stackoverflow.com/q/20469026/3542091 -->
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>log4j-over-slf4j</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hdfs</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>${hadoop.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
+ </exclusions>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+</project>
[13/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
deleted file mode 100644
index 2aa98ba..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.kafka;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.kafka.ZkHosts;
-import org.apache.storm.kafka.trident.OpaqueTridentKafkaSpout;
-import org.apache.storm.kafka.trident.TridentKafkaConfig;
-import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
-import org.apache.storm.kafka.trident.TridentKafkaUpdater;
-import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Create a Kafka spout/sink based on the URI and properties. The URI has the format of
- * kafka://zkhost:port/broker_path?topic=topic. The properties are in JSON format which specifies the producer config
- * of the Kafka broker.
- */
-public class KafkaDataSourcesProvider implements DataSourcesProvider {
- private static final int DEFAULT_ZK_PORT = 2181;
-
- private static class SqlKafkaMapper implements TridentTupleToKafkaMapper<Object, ByteBuffer> {
- private final int primaryKeyIndex;
- private final IOutputSerializer serializer;
-
- private SqlKafkaMapper(int primaryKeyIndex, IOutputSerializer serializer) {
- this.primaryKeyIndex = primaryKeyIndex;
- this.serializer = serializer;
- }
-
- @Override
- public Object getKeyFromTuple(TridentTuple tuple) {
- return tuple.get(primaryKeyIndex);
- }
-
- @Override
- public ByteBuffer getMessageFromTuple(TridentTuple tuple) {
- return serializer.write(tuple.getValues(), null);
- }
- }
-
- private static class KafkaTridentDataSource implements ISqlTridentDataSource {
- private final TridentKafkaConfig conf;
- private final String topic;
- private final int primaryKeyIndex;
- private final Properties props;
- private final IOutputSerializer serializer;
- private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
- Properties props, IOutputSerializer serializer) {
- this.conf = conf;
- this.topic = topic;
- this.primaryKeyIndex = primaryKeyIndex;
- this.props = props;
- this.serializer = serializer;
- }
-
- @Override
- public ITridentDataSource getProducer() {
- return new OpaqueTridentKafkaSpout(conf);
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- Preconditions.checkArgument(!props.isEmpty(),
- "Writable Kafka Table " + topic + " must contain producer config");
- HashMap<String, Object> producerConfig = (HashMap<String, Object>) props.get("producer");
- props.putAll(producerConfig);
- Preconditions.checkState(props.containsKey("bootstrap.servers"),
- "Writable Kafka Table " + topic + " must contain \"bootstrap.servers\" config");
-
- SqlKafkaMapper mapper = new SqlKafkaMapper(primaryKeyIndex, serializer);
-
- TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
- .withKafkaTopicSelector(new DefaultTopicSelector(topic))
- .withProducerProperties(props)
- .withTridentTupleToKafkaMapper(mapper);
-
- TridentKafkaUpdater stateUpdater = new TridentKafkaUpdater();
-
- return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
- }
- }
-
- @Override
- public String scheme() {
- return "kafka";
- }
-
- @Override
- public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_ZK_PORT;
- ZkHosts zk = new ZkHosts(uri.getHost() + ":" + port, uri.getPath());
- Map<String, String> values = parseURIParams(uri.getQuery());
- String topic = values.get("topic");
- Preconditions.checkNotNull(topic, "No topic of the spout is specified");
- TridentKafkaConfig conf = new TridentKafkaConfig(zk, topic);
- 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);
- conf.scheme = new SchemeAsMultiScheme(scheme);
- IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-
- return new KafkaTridentDataSource(conf, topic, primaryIndex, properties, serializer);
- }
-
- private static Map<String, String> parseURIParams(String query) {
- HashMap<String, String> res = new HashMap<>();
- if (query == null) {
- return res;
- }
-
- String[] params = query.split("&");
- for (String p : params) {
- String[] v = p.split("=", 2);
- if (v.length > 1) {
- res.put(v[0], v[1]);
- }
- }
- return res;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 7f687cc..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.storm.sql.kafka.KafkaDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
deleted file mode 100644
index 0cde492..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.kafka;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.kafka.trident.TridentKafkaState;
-import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
-import org.apache.storm.kafka.trident.TridentKafkaUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.Future;
-
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.argThat;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
-public class TestKafkaDataSourcesProvider {
- private static final List<FieldInfo> FIELDS = ImmutableList.of(
- new FieldInfo("ID", int.class, true),
- new FieldInfo("val", String.class, false));
- private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
- private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
- private static final Properties TBL_PROPERTIES = new Properties();
-
- static {
- Map<String,Object> map = new HashMap<>();
- map.put("bootstrap.servers", "localhost:9092");
- map.put("acks", "1");
- map.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- map.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
- TBL_PROPERTIES.put("producer", map);
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testKafkaSink() {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create("kafka://mock?topic=foo"), null, null, TBL_PROPERTIES, FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(TridentKafkaStateFactory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(TridentKafkaUpdater.class, consumer.getStateUpdater().getClass());
-
- TridentKafkaState state = (TridentKafkaState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
- KafkaProducer producer = mock(KafkaProducer.class);
- doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class));
- Whitebox.setInternalState(state, "producer", producer);
-
- List<TridentTuple> tupleList = mockTupleList();
- for (TridentTuple t : tupleList) {
- state.updateState(Collections.singletonList(t), null);
- verify(producer).send(argThat(new KafkaMessageMatcher(t)));
- }
- verifyNoMoreInteractions(producer);
- }
-
- private static List<TridentTuple> mockTupleList() {
- List<TridentTuple> tupleList = new ArrayList<>();
- TridentTuple t0 = mock(TridentTuple.class);
- TridentTuple t1 = mock(TridentTuple.class);
- doReturn(1).when(t0).get(0);
- doReturn(2).when(t1).get(0);
- doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
- doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
- tupleList.add(t0);
- tupleList.add(t1);
- return tupleList;
- }
-
- private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
- private static final int PRIMARY_INDEX = 0;
- private final TridentTuple tuple;
-
- private KafkaMessageMatcher(TridentTuple tuple) {
- this.tuple = tuple;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public boolean matches(Object o) {
- ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
- if (m.key() != tuple.get(PRIMARY_INDEX)) {
- return false;
- }
- ByteBuffer buf = m.value();
- ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
- return b.equals(buf);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml b/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
deleted file mode 100644
index 40fde8a..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
+++ /dev/null
@@ -1,84 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-mongodb</artifactId>
-
- <developers>
- <developer>
- <id>vesense</id>
- <name>Xin Wang</name>
- <email>data.xinwang@gmail.com</email>
- </developer>
- </developers>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-mongodb</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/resources</directory>
- </resource>
- </resources>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
deleted file mode 100644
index 60d52d1..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.mongodb;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.mongodb.common.mapper.MongoMapper;
-import org.apache.storm.mongodb.trident.state.MongoState;
-import org.apache.storm.mongodb.trident.state.MongoStateFactory;
-import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.tuple.ITuple;
-import org.bson.Document;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a MongoDB sink based on the URI and properties. The URI has the format of
- * mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]][/[database][?options]].
- * The properties are in JSON format which specifies the name of the MongoDB collection and etc.
- */
-public class MongoDataSourcesProvider implements DataSourcesProvider {
-
- private static class MongoTridentDataSource implements ISqlTridentDataSource {
- private final String url;
- private final Properties props;
- private final IOutputSerializer serializer;
-
- private MongoTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
- this.url = url;
- this.props = props;
- this.serializer = serializer;
- }
-
- @Override
- public ITridentDataSource getProducer() {
- throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- Preconditions.checkArgument(!props.isEmpty(), "Writable MongoDB must contain collection config");
- String serField = props.getProperty("trident.ser.field", "tridentSerField");
- MongoMapper mapper = new TridentMongoMapper(serField, serializer);
-
- MongoState.Options options = new MongoState.Options()
- .withUrl(url)
- .withCollectionName(props.getProperty("collection.name"))
- .withMapper(mapper);
-
- StateFactory stateFactory = new MongoStateFactory(options);
- StateUpdater stateUpdater = new MongoStateUpdater();
-
- return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
- }
- }
-
- private static class TridentMongoMapper implements MongoMapper {
- private final String serField;
- private final IOutputSerializer serializer;
-
- private TridentMongoMapper(String serField, IOutputSerializer serializer) {
- this.serField = serField;
- this.serializer = serializer;
- }
-
- @Override
- public Document toDocument(ITuple tuple) {
- Document document = new Document();
- byte[] array = serializer.write(tuple.getValues(), null).array();
- document.append(serField, array);
- return document;
- }
-
- @Override
- public Document toDocumentByKeys(List<Object> keys) {
- return null;
- }
- }
-
- @Override
- public String scheme() {
- return "mongodb";
- }
-
- @Override
- public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
- IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
- return new MongoTridentDataSource(uri.toString(), properties, serializer);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index e46d794..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.storm.sql.mongodb.MongoDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
deleted file mode 100644
index 3b15345..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.mongodb;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.mongodb.common.MongoDBClient;
-import org.apache.storm.mongodb.trident.state.MongoState;
-import org.apache.storm.mongodb.trident.state.MongoStateFactory;
-import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.bson.Document;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
-public class TestMongoDataSourcesProvider {
- private static final List<FieldInfo> FIELDS = ImmutableList.of(
- new FieldInfo("ID", int.class, true),
- new FieldInfo("val", String.class, false));
- private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
- private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
- private static final Properties TBL_PROPERTIES = new Properties();
-
- static {
- TBL_PROPERTIES.put("collection.name", "collection1");
- TBL_PROPERTIES.put("trident.ser.field", "tridentSerField");
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testMongoSink() {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create("mongodb://127.0.0.1:27017/test"), null, null, TBL_PROPERTIES, FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(MongoStateFactory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(MongoStateUpdater.class, consumer.getStateUpdater().getClass());
-
- MongoState state = (MongoState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
- StateUpdater stateUpdater = consumer.getStateUpdater();
-
- MongoDBClient mongoClient = mock(MongoDBClient.class);
- Whitebox.setInternalState(state, "mongoClient", mongoClient);
-
- List<TridentTuple> tupleList = mockTupleList();
-
- for (TridentTuple t : tupleList) {
- stateUpdater.updateState(state, Collections.singletonList(t), null);
- verify(mongoClient).insert(argThat(new MongoArgMatcher(t)) , eq(true));
- }
-
- verifyNoMoreInteractions(mongoClient);
- }
-
- private static List<TridentTuple> mockTupleList() {
- List<TridentTuple> tupleList = new ArrayList<>();
- TridentTuple t0 = mock(TridentTuple.class);
- TridentTuple t1 = mock(TridentTuple.class);
- doReturn(1).when(t0).get(0);
- doReturn(2).when(t1).get(0);
- doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
- doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
- tupleList.add(t0);
- tupleList.add(t1);
- return tupleList;
- }
-
- private static class MongoArgMatcher extends ArgumentMatcher<List<Document>> {
- private final TridentTuple tuple;
-
- private MongoArgMatcher(TridentTuple tuple) {
- this.tuple = tuple;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public boolean matches(Object o) {
- Document doc = ((List<Document>)o).get(0);
- ByteBuffer buf = ByteBuffer.wrap((byte[])doc.get(TBL_PROPERTIES.getProperty("trident.ser.field")));
- ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
- return b.equals(buf);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/pom.xml b/external/sql/storm-sql-external/storm-sql-redis/pom.xml
deleted file mode 100644
index 19cc699..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/pom.xml
+++ /dev/null
@@ -1,76 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-redis</artifactId>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-redis</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/resources</directory>
- </resource>
- </resources>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
deleted file mode 100644
index 68933b2..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.redis;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.redis.common.config.JedisClusterConfig;
-import org.apache.storm.redis.common.config.JedisPoolConfig;
-import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
-import org.apache.storm.redis.common.mapper.RedisStoreMapper;
-import org.apache.storm.redis.trident.state.RedisClusterState;
-import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
-import org.apache.storm.redis.trident.state.RedisState;
-import org.apache.storm.redis.trident.state.RedisStateUpdater;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.tuple.ITuple;
-import redis.clients.util.JedisURIHelper;
-
-import java.io.Serializable;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a Redis sink based on the URI and properties. The URI has the format of
- * redis://:[password]@[host]:[port]/[dbIdx]. Only host is mandatory and others can be set to default.
- *
- * The properties are in JSON format which specifies the config of the Redis data type and etc.
- * Please note that when "use.redis.cluster" is "true", cluster discovery is only done from given URI.
- */
-public class RedisDataSourcesProvider implements DataSourcesProvider {
- private static final int DEFAULT_REDIS_PORT = 6379;
- private static final int DEFAULT_TIMEOUT = 2000;
-
- private abstract static class AbstractRedisTridentDataSource implements ISqlTridentDataSource, Serializable {
- protected abstract StateFactory newStateFactory();
- protected abstract StateUpdater newStateUpdater(RedisStoreMapper storeMapper);
-
- private final Properties props;
- private final List<FieldInfo> fields;
- private final IOutputSerializer serializer;
-
- AbstractRedisTridentDataSource(Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
- this.props = props;
- this.fields = fields;
- this.serializer = serializer;
- }
-
- @Override
- public ITridentDataSource getProducer() {
- throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- RedisDataTypeDescription dataTypeDescription = getDataTypeDesc(props);
-
- RedisStoreMapper storeMapper = new TridentRedisStoreMapper(dataTypeDescription, fields, serializer);
-
- StateFactory stateFactory = newStateFactory();
- StateUpdater stateUpdater = newStateUpdater(storeMapper);
-
- return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
- }
-
- private RedisDataTypeDescription getDataTypeDesc(Properties props) {
- Preconditions.checkArgument(props.containsKey("data.type"),
- "Redis data source must contain \"data.type\" config");
-
- RedisDataTypeDescription.RedisDataType dataType = RedisDataTypeDescription.RedisDataType.valueOf(props.getProperty("data.type").toUpperCase());
- String additionalKey = props.getProperty("data.additional.key");
-
- return new RedisDataTypeDescription(dataType, additionalKey);
- }
- }
-
- private static class RedisClusterTridentDataSource extends AbstractRedisTridentDataSource {
- private final JedisClusterConfig config;
-
- RedisClusterTridentDataSource(JedisClusterConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
- super(props, fields, serializer);
- this.config = config;
- }
-
- @Override
- protected StateFactory newStateFactory() {
- return new RedisClusterState.Factory(config);
- }
-
- @Override
- protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
- return new RedisClusterStateUpdater(storeMapper);
- }
- }
-
- private static class RedisTridentDataSource extends AbstractRedisTridentDataSource {
- private final JedisPoolConfig config;
-
- RedisTridentDataSource(JedisPoolConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
- super(props, fields, serializer);
- this.config = config;
- }
-
- @Override
- protected StateFactory newStateFactory() {
- return new RedisState.Factory(config);
- }
-
- @Override
- protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
- return new RedisStateUpdater(storeMapper);
- }
- }
-
- @Override
- public String scheme() {
- return "redis";
- }
-
- @Override
- public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties props, List<FieldInfo> fields) {
- Preconditions.checkArgument(JedisURIHelper.isValid(uri), "URI is not valid for Redis: " + uri);
-
- String host = uri.getHost();
- int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_REDIS_PORT;
- int dbIdx = JedisURIHelper.getDBIndex(uri);
- String password = JedisURIHelper.getPassword(uri);
-
- int timeout = Integer.parseInt(props.getProperty("redis.timeout", String.valueOf(DEFAULT_TIMEOUT)));
-
- boolean clusterMode = Boolean.valueOf(props.getProperty("use.redis.cluster", "false"));
-
- List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
- IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, props, fieldNames);
- if (clusterMode) {
- JedisClusterConfig config = new JedisClusterConfig.Builder()
- .setNodes(Collections.singleton(new InetSocketAddress(host, port)))
- .setTimeout(timeout)
- .build();
- return new RedisClusterTridentDataSource(config, props, fields, serializer);
- } else {
- JedisPoolConfig config = new JedisPoolConfig(host, port, timeout, password, dbIdx);
- return new RedisTridentDataSource(config, props, fields, serializer);
- }
- }
-
- private static class TridentRedisStoreMapper implements RedisStoreMapper {
- private final RedisDataTypeDescription dataTypeDescription;
- private final FieldInfo primaryKeyField;
- private final IOutputSerializer outputSerializer;
-
- private TridentRedisStoreMapper(RedisDataTypeDescription dataTypeDescription, List<FieldInfo> fields, IOutputSerializer outputSerializer) {
- this.dataTypeDescription = dataTypeDescription;
- this.outputSerializer = outputSerializer;
-
- // find primary key from constructor
- FieldInfo pkField = findPrimaryKeyField(fields);
- Preconditions.checkArgument(pkField != null, "Primary key must be presented to field list");
-
- this.primaryKeyField = pkField;
- }
-
- 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 RedisDataTypeDescription getDataTypeDescription() {
- return dataTypeDescription;
- }
-
- @Override
- public String getKeyFromTuple(ITuple tuple) {
- String keyFieldName = primaryKeyField.name();
- Object key = tuple.getValueByField(keyFieldName);
- if (key == null) {
- throw new NullPointerException("key field " + keyFieldName + " is null");
- }
- return String.valueOf(key);
- }
-
- @Override
- public String getValueFromTuple(ITuple tuple) {
- byte[] array = outputSerializer.write(tuple.getValues(), null).array();
- return new String(array);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 23b0444..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.storm.sql.redis.RedisDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
deleted file mode 100644
index 94d4949..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.redis;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.redis.trident.state.RedisClusterState;
-import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
-import org.apache.storm.redis.trident.state.RedisState;
-import org.apache.storm.redis.trident.state.RedisStateUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-import redis.clients.jedis.Jedis;
-import redis.clients.jedis.JedisCluster;
-import redis.clients.jedis.JedisPool;
-import redis.clients.jedis.Pipeline;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class TestRedisDataSourcesProvider {
- private static final List<FieldInfo> FIELDS = ImmutableList.of(
- new FieldInfo("ID", int.class, true),
- new FieldInfo("val", String.class, false));
- private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
- private static final String ADDITIONAL_KEY = "hello";
- private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
- private static final Properties TBL_PROPERTIES = new Properties();
- private static final Properties CLUSTER_TBL_PROPERTIES = new Properties();
-
- static {
- TBL_PROPERTIES.put("data.type", "HASH");
- TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
- CLUSTER_TBL_PROPERTIES.put("data.type", "HASH");
- CLUSTER_TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
- CLUSTER_TBL_PROPERTIES.put("use.redis.cluster", "true");
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testRedisSink() {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create("redis://:foobared@localhost:6380/2"), null, null, TBL_PROPERTIES, FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(RedisState.Factory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(RedisStateUpdater.class, consumer.getStateUpdater().getClass());
-
- RedisState state = (RedisState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
- StateUpdater stateUpdater = consumer.getStateUpdater();
-
- JedisPool mockJedisPool = mock(JedisPool.class);
- Jedis mockJedis = mock(Jedis.class);
- Pipeline mockPipeline = mock(Pipeline.class);
-
- Whitebox.setInternalState(state, "jedisPool", mockJedisPool);
- when(mockJedisPool.getResource()).thenReturn(mockJedis);
- when(mockJedis.pipelined()).thenReturn(mockPipeline);
-
- List<TridentTuple> tupleList = mockTupleList();
-
- stateUpdater.updateState(state, tupleList, null);
- for (TridentTuple t : tupleList) {
- // PK goes to the key
- String id = String.valueOf(t.getValueByField("ID"));
- String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
- verify(mockPipeline).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
- }
-
- verify(mockPipeline).sync();
- verify(mockJedis).close();
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testRedisClusterSink() throws IOException {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create("redis://localhost:6380"), null, null, CLUSTER_TBL_PROPERTIES, FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(RedisClusterState.Factory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(RedisClusterStateUpdater.class, consumer.getStateUpdater().getClass());
-
- RedisClusterState state = (RedisClusterState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
- StateUpdater stateUpdater = consumer.getStateUpdater();
-
- JedisCluster mockJedisCluster = mock(JedisCluster.class);
-
- Whitebox.setInternalState(state, "jedisCluster", mockJedisCluster);
-
- List<TridentTuple> tupleList = mockTupleList();
-
- stateUpdater.updateState(state, tupleList, null);
- for (TridentTuple t : tupleList) {
- // PK goes to the key
- String id = String.valueOf(t.getValueByField("ID"));
- String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
- verify(mockJedisCluster).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
- }
-
- verify(mockJedisCluster, never()).close();
- }
-
- private static List<TridentTuple> mockTupleList() {
- List<TridentTuple> tupleList = new ArrayList<>();
- TridentTuple t0 = mock(TridentTuple.class);
- TridentTuple t1 = mock(TridentTuple.class);
- when(t0.getValueByField("ID")).thenReturn(1);
- when(t0.getValueByField("val")).thenReturn("2");
- doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-
- when(t1.getValueByField("ID")).thenReturn(2);
- when(t1.getValueByField("val")).thenReturn("3");
- doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-
- tupleList.add(t0);
- tupleList.add(t1);
- return tupleList;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/pom.xml b/external/sql/storm-sql-runtime/pom.xml
deleted file mode 100644
index 4666df4..0000000
--- a/external/sql/storm-sql-runtime/pom.xml
+++ /dev/null
@@ -1,136 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-runtime</artifactId>
-
- <developers>
- <developer>
- <id>haohui</id>
- <name>Haohui Mai</name>
- <email>ricetons@gmail.com</email>
- </developer>
- </developers>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.calcite</groupId>
- <artifactId>calcite-core</artifactId>
- <version>${calcite.version}</version>
- <exclusions>
- <exclusion>
- <groupId>commons-dbcp</groupId>
- <artifactId>commons-dbcp</artifactId>
- </exclusion>
- <exclusion>
- <groupId>com.google.code.findbugs</groupId>
- <artifactId>jsr305</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.pentaho</groupId>
- <artifactId>pentaho-aggdesigner-algorithm</artifactId>
- </exclusion>
- <exclusion>
- <groupId>com.fasterxml.jackson.core</groupId>
- <artifactId>jackson-annotations</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>com.fasterxml.jackson.core</groupId>
- <artifactId>jackson-databind</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.avro</groupId>
- <artifactId>avro</artifactId>
- <version>1.7.7</version>
- </dependency>
- <dependency>
- <groupId>commons-lang</groupId>
- <artifactId>commons-lang</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-csv</artifactId>
- <version>1.4</version>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/resources</directory>
- </resource>
- </resources>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <executions>
- <execution>
- <goals>
- <goal>test-jar</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>appassembler-maven-plugin</artifactId>
- <version>1.9</version>
- <executions>
- <execution>
- <id>create-repo</id>
- <goals>
- <goal>create-repository</goal>
- </goals>
- <configuration>
- <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
- <repositoryLayout>flat</repositoryLayout>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
deleted file mode 100644
index aa7e435..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.interpreter;
-
-import org.apache.calcite.DataContext;
-
-import java.io.Serializable;
-
-/**
- * This is a hack to use Calcite Context.
- */
-public class StormContext extends Context implements Serializable {
- public StormContext(DataContext root) {
- super(root);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
deleted file mode 100644
index 64be39d..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public abstract class AbstractChannelHandler implements ChannelHandler {
- @Override
- public abstract void dataReceived(ChannelContext ctx, Values data);
-
- @Override
- public void channelInactive(ChannelContext ctx) {
-
- }
-
- @Override
- public void exceptionCaught(Throwable cause) {
-
- }
-
- @Override
- public void flush(ChannelContext ctx) {
- ctx.flush();
- }
-
- @Override
- public void setSource(ChannelContext ctx, Object source) {
-
- }
-
- public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() {
- @Override
- public void dataReceived(ChannelContext ctx, Values data) {
- ctx.emit(data);
- }
- };
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
deleted file mode 100644
index 5b18b5b..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-
-import java.util.Map;
-
-/**
- * Subclass of AbstractTupleProcessor provides a series of tuple. It
- * takes a series of iterators of {@link Values} and produces a stream of
- * tuple.
- *
- * The subclass implements the {@see next()} method to provide
- * the output of the stream. It can choose to return null in {@see next()} to
- * indicate that this particular iteration is a no-op. SQL processors depend
- * on this semantic to implement filtering and nullable records.
- */
-public abstract class AbstractValuesProcessor {
-
- /**
- * Initialize the data sources.
- *
- * @param data a map from the table name to the iterators of the values.
- *
- */
- public abstract void initialize(Map<String, DataSource> data, ChannelHandler
- result);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
deleted file mode 100644
index 65ad01c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public interface ChannelContext {
- /**
- * Emit data to the next stage of the data pipeline.
- */
- void emit(Values data);
- void fireChannelInactive();
- void flush();
- void setSource(Object source);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
deleted file mode 100644
index af02b7e..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-/**
- * DataListener provides an event-driven interface for the user to process
- * series of events.
- */
-public interface ChannelHandler {
- void dataReceived(ChannelContext ctx, Values data);
-
- /**
- * The producer of the data has indicated that the channel is no longer
- * active.
- * @param ctx
- */
- void channelInactive(ChannelContext ctx);
-
- void exceptionCaught(Throwable cause);
-
- void flush(ChannelContext ctx);
-
- void setSource(ChannelContext ctx, Object source);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
deleted file mode 100644
index 3b5eedd..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public class Channels {
- private static final ChannelContext VOID_CTX = new ChannelContext() {
- @Override
- public void emit(Values data) {}
-
- @Override
- public void fireChannelInactive() {}
-
- @Override
- public void flush() {
-
- }
-
- @Override
- public void setSource(java.lang.Object source) {
-
- }
- };
-
- private static class ChannelContextAdapter implements ChannelContext {
- private final ChannelHandler handler;
- private final ChannelContext next;
-
- public ChannelContextAdapter(
- ChannelContext next, ChannelHandler handler) {
- this.handler = handler;
- this.next = next;
- }
-
- @Override
- public void emit(Values data) {
- handler.dataReceived(next, data);
- }
-
- @Override
- public void fireChannelInactive() {
- handler.channelInactive(next);
- }
-
- @Override
- public void flush() {
- handler.flush(next);
- }
-
- @Override
- public void setSource(java.lang.Object source) {
- handler.setSource(next, source);
- next.setSource(source); // propagate through the chain
- }
- }
-
- private static class ForwardingChannelContext implements ChannelContext {
- private final ChannelContext next;
-
- public ForwardingChannelContext(ChannelContext next) {
- this.next = next;
- }
-
- @Override
- public void emit(Values data) {
- next.emit(data);
- }
-
- @Override
- public void fireChannelInactive() {
- next.fireChannelInactive();
- }
-
- @Override
- public void flush() {
- next.flush();
- }
-
- @Override
- public void setSource(Object source) {
- next.setSource(source);
- }
- }
-
- public static ChannelContext chain(
- ChannelContext next, ChannelHandler handler) {
- return new ChannelContextAdapter(next, handler);
- }
-
- public static ChannelContext voidContext() {
- return VOID_CTX;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
deleted file mode 100644
index 352af73..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-/**
- * A DataSource ingests data in StormSQL. It provides a series of tuple to
- * the downstream {@link ChannelHandler}.
- *
- */
-public interface DataSource {
- void open(ChannelContext ctx);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
deleted file mode 100644
index dbece9c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-public interface DataSourcesProvider {
- /**
- * @return the scheme of the data source
- */
- String scheme();
-
- /**
- * Construct a new data source.
- * @param uri The URI that specifies the data source. The format of the URI
- * is fully customizable.
- * @param inputFormatClass the name of the class that deserializes data.
- * It is null when unspecified.
- * @param outputFormatClass the name of the class that serializes data. It
- * is null when unspecified.
- * @param fields The name of the fields and the schema of the table.
- */
- DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields);
-
- ISqlTridentDataSource constructTrident(
- URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
deleted file mode 100644
index dfefb01..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.ServiceLoader;
-
-public class DataSourcesRegistry {
- private static final Logger LOG = LoggerFactory.getLogger(
- DataSourcesRegistry.class);
- private static final Map<String, DataSourcesProvider> providers;
-
- static {
- providers = new HashMap<>();
- ServiceLoader<DataSourcesProvider> loader = ServiceLoader.load(
- DataSourcesProvider.class);
- for (DataSourcesProvider p : loader) {
- LOG.info("Registering scheme {} with {}", p.scheme(), p);
- providers.put(p.scheme(), p);
- }
- }
-
- private DataSourcesRegistry() {
- }
-
- public static DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- DataSourcesProvider provider = providers.get(uri.getScheme());
- if (provider == null) {
- return null;
- }
-
- return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
- }
-
- public static ISqlTridentDataSource constructTridentDataSource(
- URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- DataSourcesProvider provider = providers.get(uri.getScheme());
- if (provider == null) {
- return null;
- }
-
- return provider.constructTrident(uri, inputFormatClass, outputFormatClass, properties, fields);
- }
-
- /**
- * Allow unit tests to inject data sources.
- */
- public static Map<String, DataSourcesProvider> providerMap() {
- return providers;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
deleted file mode 100644
index 03b030b..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import java.io.Serializable;
-
-/**
- * Describe each column of the field
- */
-public class FieldInfo implements Serializable {
- private final String name;
- private final Class<?> type;
- private final boolean isPrimary;
-
- public FieldInfo(String name, Class<?> type, boolean isPrimary) {
- this.name = name;
- this.type = type;
- this.isPrimary = isPrimary;
- }
-
- public String name() {
- return name;
- }
-
- public Class<?> type() {
- return type;
- }
-
- public boolean isPrimary() {
- return isPrimary;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
deleted file mode 100644
index b6670d9..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-public interface IOutputSerializer {
- /**
- * Serialize the data to a ByteBuffer. The caller can pass in a ByteBuffer so that the serializer can reuse the
- * memory.
- *
- * @return A ByteBuffer contains the serialized result.
- */
- ByteBuffer write(List<Object> data, ByteBuffer buffer);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
deleted file mode 100644
index 9eae5ae..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-/**
- * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
- */
-public interface ISqlTridentDataSource {
- /**
- * SqlTridentConsumer is a data structure containing StateFactory and StateUpdater for consuming tuples with State.
- *
- * Please note that StateFactory and StateUpdater should use same class which implements State.
- *
- * @see org.apache.storm.trident.state.StateFactory
- * @see org.apache.storm.trident.state.StateUpdater
- */
- interface SqlTridentConsumer {
- StateFactory getStateFactory();
- StateUpdater getStateUpdater();
- }
-
- /**
- * Provides instance of ITridentDataSource which can be used as producer in Trident.
- *
- * Since ITridentDataSource is a marker interface for Trident Spout interfaces, this method should effectively
- * return an instance of one of these interfaces (can be changed if Trident API evolves) or descendants:
- * - IBatchSpout
- * - ITridentSpout
- * - IPartitionedTridentSpout
- * - IOpaquePartitionedTridentSpout
- *
- * @see org.apache.storm.trident.spout.ITridentDataSource
- * @see org.apache.storm.trident.spout.IBatchSpout
- * @see org.apache.storm.trident.spout.ITridentSpout
- * @see org.apache.storm.trident.spout.IPartitionedTridentSpout
- * @see org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout
- */
- ITridentDataSource getProducer();
-
- /**
- * Provides instance of SqlTridentConsumer which can be used as consumer (State) in Trident.
- *
- * @see SqlTridentConsumer
- */
- SqlTridentConsumer getConsumer();
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
deleted file mode 100644
index c9abd16..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-public class SimpleSqlTridentConsumer implements ISqlTridentDataSource.SqlTridentConsumer {
- private final StateFactory stateFactory;
- private final StateUpdater stateUpdater;
-
- public SimpleSqlTridentConsumer(StateFactory stateFactory, StateUpdater stateUpdater) {
- this.stateFactory = stateFactory;
- this.stateUpdater = stateUpdater;
- }
-
- @Override
- public StateFactory getStateFactory() {
- return stateFactory;
- }
-
- @Override
- public StateUpdater getStateUpdater() {
- return stateUpdater;
- }
-}
[22/23] storm git commit: Merge branch 'STORM-2453' of
https://github.com/HeartSaVioR/storm into STORM-2453-merge
Posted by ka...@apache.org.
Merge branch 'STORM-2453' of https://github.com/HeartSaVioR/storm into STORM-2453-merge
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1dd4bbc4
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1dd4bbc4
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1dd4bbc4
Branch: refs/heads/master
Commit: 1dd4bbc4874f38eee0e96eb19a342cc1217baab1
Parents: e84b39d e9d7833
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Apr 6 08:11:02 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Apr 6 08:11:02 2017 +0900
----------------------------------------------------------------------
bin/storm-kafka-monitor | 2 +-
bin/storm.py | 9 +-
examples/flux-examples/README.md | 93 --
examples/flux-examples/pom.xml | 152 ----
.../flux/examples/StatefulWordCounter.java | 64 --
.../storm/flux/examples/TestPrintBolt.java | 39 -
.../storm/flux/examples/TestWindowBolt.java | 47 -
.../storm/flux/examples/WordCountClient.java | 74 --
.../apache/storm/flux/examples/WordCounter.java | 71 --
.../src/main/resources/hbase_bolt.properties | 18 -
.../src/main/resources/hdfs_bolt.properties | 26 -
.../src/main/resources/kafka_spout.yaml | 136 ---
.../src/main/resources/multilang.yaml | 89 --
.../src/main/resources/simple_hbase.yaml | 92 --
.../src/main/resources/simple_hdfs.yaml | 105 ---
.../resources/simple_stateful_wordcount.yaml | 60 --
.../src/main/resources/simple_windowing.yaml | 69 --
.../src/main/resources/simple_wordcount.yaml | 68 --
external/flux/README.md | 872 -------------------
external/flux/flux-core/pom.xml | 112 ---
.../main/java/org/apache/storm/flux/Flux.java | 261 ------
.../java/org/apache/storm/flux/FluxBuilder.java | 630 --------------
.../apache/storm/flux/api/TopologySource.java | 39 -
.../org/apache/storm/flux/model/BeanDef.java | 39 -
.../storm/flux/model/BeanListReference.java | 37 -
.../apache/storm/flux/model/BeanReference.java | 39 -
.../org/apache/storm/flux/model/BoltDef.java | 24 -
.../storm/flux/model/ConfigMethodDef.java | 67 --
.../storm/flux/model/ExecutionContext.java | 77 --
.../apache/storm/flux/model/GroupingDef.java | 77 --
.../org/apache/storm/flux/model/IncludeDef.java | 54 --
.../org/apache/storm/flux/model/ObjectDef.java | 95 --
.../apache/storm/flux/model/PropertyDef.java | 58 --
.../org/apache/storm/flux/model/SpoutDef.java | 24 -
.../org/apache/storm/flux/model/StreamDef.java | 64 --
.../apache/storm/flux/model/TopologyDef.java | 216 -----
.../storm/flux/model/TopologySourceDef.java | 36 -
.../org/apache/storm/flux/model/VertexDef.java | 36 -
.../apache/storm/flux/parser/FluxParser.java | 208 -----
.../flux-core/src/main/resources/splash.txt | 9 -
.../org/apache/storm/flux/FluxBuilderTest.java | 31 -
.../org/apache/storm/flux/IntegrationTest.java | 39 -
.../java/org/apache/storm/flux/TCKTest.java | 256 ------
.../multilang/MultilangEnvirontmentTest.java | 89 --
.../apache/storm/flux/test/SimpleTopology.java | 59 --
.../storm/flux/test/SimpleTopologySource.java | 52 --
.../test/SimpleTopologyWithConfigParam.java | 55 --
.../org/apache/storm/flux/test/TestBolt.java | 131 ---
.../storm/flux/test/TridentTopologySource.java | 71 --
.../src/test/resources/configs/bad_hbase.yaml | 98 ---
.../test/resources/configs/bad_shell_test.yaml | 118 ---
.../resources/configs/config-methods-test.yaml | 92 --
.../resources/configs/diamond-topology.yaml | 87 --
.../existing-topology-method-override.yaml | 25 -
.../existing-topology-reflection-config.yaml | 24 -
.../configs/existing-topology-reflection.yaml | 24 -
.../configs/existing-topology-trident.yaml | 24 -
.../resources/configs/existing-topology.yaml | 23 -
.../src/test/resources/configs/hdfs_test.yaml | 97 ---
.../test/resources/configs/include_test.yaml | 25 -
.../configs/invalid-existing-topology.yaml | 33 -
.../src/test/resources/configs/kafka_test.yaml | 126 ---
.../src/test/resources/configs/shell_test.yaml | 118 ---
.../test/resources/configs/simple_hbase.yaml | 120 ---
.../resources/configs/substitution-test.yaml | 106 ---
.../src/test/resources/configs/tck.yaml | 95 --
.../src/test/resources/configs/test.properties | 18 -
.../flux-core/src/test/resources/log4j2.xml | 34 -
external/flux/flux-ui/README.md | 3 -
external/flux/flux-wrappers/pom.xml | 51 --
.../flux/wrappers/bolts/FluxShellBolt.java | 175 ----
.../storm/flux/wrappers/bolts/LogInfoBolt.java | 44 -
.../flux/wrappers/spouts/FluxShellSpout.java | 175 ----
.../main/resources/resources/randomsentence.js | 93 --
.../main/resources/resources/splitsentence.py | 24 -
external/flux/pom.xml | 64 --
external/sql/README.md | 207 -----
external/sql/pom.xml | 47 -
external/sql/storm-sql-core/pom.xml | 279 ------
.../sql/storm-sql-core/src/codegen/config.fmpp | 23 -
.../storm-sql-core/src/codegen/data/Parser.tdd | 80 --
.../src/codegen/includes/license.ftl | 17 -
.../src/codegen/includes/parserImpls.ftl | 113 ---
.../storm/sql/AbstractTridentProcessor.java | 58 --
.../src/jvm/org/apache/storm/sql/StormSql.java | 59 --
.../jvm/org/apache/storm/sql/StormSqlImpl.java | 290 ------
.../org/apache/storm/sql/StormSqlRunner.java | 85 --
.../sql/calcite/ParallelStreamableTable.java | 35 -
.../apache/storm/sql/compiler/CompilerUtil.java | 183 ----
.../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 -----
.../sql/compiler/StormSqlTypeFactoryImpl.java | 51 --
.../standalone/BuiltinAggregateFunctions.java | 238 -----
.../backends/standalone/PlanCompiler.java | 139 ---
.../standalone/PostOrderRelNodeVisitor.java | 132 ---
.../backends/standalone/RelNodeCompiler.java | 484 ----------
.../storm/sql/javac/CompilingClassLoader.java | 225 -----
.../storm/sql/parser/ColumnConstraint.java | 42 -
.../storm/sql/parser/ColumnDefinition.java | 44 -
.../storm/sql/parser/SqlCreateFunction.java | 94 --
.../apache/storm/sql/parser/SqlCreateTable.java | 166 ----
.../apache/storm/sql/parser/SqlDDLKeywords.java | 27 -
.../apache/storm/sql/parser/StormParser.java | 47 -
.../apache/storm/sql/parser/UnparseUtil.java | 60 --
.../sql/planner/StormRelDataTypeSystem.java | 37 -
.../apache/storm/sql/planner/StormRelUtils.java | 68 --
.../planner/UnsupportedOperatorsVisitor.java | 24 -
.../storm/sql/planner/rel/StormCalcRelBase.java | 32 -
.../sql/planner/rel/StormFilterRelBase.java | 32 -
.../storm/sql/planner/rel/StormJoinRelBase.java | 36 -
.../sql/planner/rel/StormProjectRelBase.java | 35 -
.../storm/sql/planner/rel/StormRelNode.java | 24 -
.../planner/rel/StormStreamInsertRelBase.java | 36 -
.../sql/planner/rel/StormStreamScanRelBase.java | 34 -
.../storm/sql/planner/trident/QueryPlanner.java | 156 ----
.../sql/planner/trident/TridentPlanCreator.java | 125 ---
.../planner/trident/TridentStormRuleSets.java | 110 ---
.../sql/planner/trident/rel/TridentCalcRel.java | 97 ---
.../planner/trident/rel/TridentFilterRel.java | 65 --
.../trident/rel/TridentLogicalConvention.java | 67 --
.../planner/trident/rel/TridentProjectRel.java | 69 --
.../sql/planner/trident/rel/TridentRel.java | 26 -
.../trident/rel/TridentStreamInsertRel.java | 76 --
.../trident/rel/TridentStreamScanRel.java | 55 --
.../trident/rules/TridentAggregateRule.java | 39 -
.../planner/trident/rules/TridentCalcRule.java | 45 -
.../trident/rules/TridentFilterRule.java | 45 -
.../planner/trident/rules/TridentJoinRule.java | 37 -
.../trident/rules/TridentModifyRule.java | 71 --
.../trident/rules/TridentProjectRule.java | 45 -
.../planner/trident/rules/TridentScanRule.java | 58 --
.../test/org/apache/storm/sql/TestStormSql.java | 492 -----------
.../storm/sql/compiler/TestExprSemantic.java | 410 ---------
.../backends/standalone/TestCompilerUtils.java | 183 ----
.../backends/standalone/TestPlanCompiler.java | 104 ---
.../standalone/TestRelNodeCompiler.java | 64 --
.../backends/trident/TestCompilerUtils.java | 208 -----
.../backends/trident/TestPlanCompiler.java | 235 -----
.../apache/storm/sql/parser/TestSqlParser.java | 54 --
.../storm-sql-external/storm-sql-hdfs/pom.xml | 104 ---
.../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 ---
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/hdfs/TestHdfsDataSourcesProvider.java | 129 ---
.../storm-sql-external/storm-sql-kafka/pom.xml | 93 --
.../sql/kafka/KafkaDataSourcesProvider.java | 170 ----
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/kafka/TestKafkaDataSourcesProvider.java | 125 ---
.../storm-sql-mongodb/pom.xml | 84 --
.../sql/mongodb/MongoDataSourcesProvider.java | 126 ---
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../mongodb/TestMongoDataSourcesProvider.java | 122 ---
.../storm-sql-external/storm-sql-redis/pom.xml | 76 --
.../sql/redis/RedisDataSourcesProvider.java | 228 -----
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/redis/TestRedisDataSourcesProvider.java | 158 ----
external/sql/storm-sql-runtime/pom.xml | 136 ---
.../calcite/interpreter/StormContext.java | 31 -
.../sql/runtime/AbstractChannelHandler.java | 52 --
.../sql/runtime/AbstractValuesProcessor.java | 46 -
.../storm/sql/runtime/ChannelContext.java | 30 -
.../storm/sql/runtime/ChannelHandler.java | 41 -
.../org/apache/storm/sql/runtime/Channels.java | 109 ---
.../apache/storm/sql/runtime/DataSource.java | 27 -
.../storm/sql/runtime/DataSourcesProvider.java | 47 -
.../storm/sql/runtime/DataSourcesRegistry.java | 76 --
.../org/apache/storm/sql/runtime/FieldInfo.java | 47 -
.../storm/sql/runtime/IOutputSerializer.java | 31 -
.../sql/runtime/ISqlTridentDataSource.java | 65 --
.../sql/runtime/SimpleSqlTridentConsumer.java | 41 -
.../storm/sql/runtime/StormSqlFunctions.java | 34 -
.../calcite/DebuggableExecutableExpression.java | 45 -
.../runtime/calcite/ExecutableExpression.java | 31 -
.../sql/runtime/calcite/StormDataContext.java | 79 --
.../socket/SocketDataSourcesProvider.java | 101 ---
.../datasource/socket/trident/SocketState.java | 89 --
.../socket/trident/SocketStateUpdater.java | 59 --
.../socket/trident/TridentSocketSpout.java | 177 ----
.../sql/runtime/serde/avro/AvroScheme.java | 74 --
.../sql/runtime/serde/avro/AvroSerializer.java | 72 --
.../sql/runtime/serde/avro/CachedSchemas.java | 41 -
.../storm/sql/runtime/serde/csv/CsvScheme.java | 70 --
.../sql/runtime/serde/csv/CsvSerializer.java | 59 --
.../sql/runtime/serde/json/JsonScheme.java | 58 --
.../sql/runtime/serde/json/JsonSerializer.java | 57 --
.../storm/sql/runtime/serde/tsv/TsvScheme.java | 58 --
.../sql/runtime/serde/tsv/TsvSerializer.java | 54 --
.../trident/functions/EvaluationCalc.java | 86 --
.../trident/functions/EvaluationFilter.java | 61 --
.../trident/functions/EvaluationFunction.java | 67 --
.../trident/functions/ForwardFunction.java | 30 -
.../storm/sql/runtime/utils/FieldInfoUtils.java | 39 -
.../storm/sql/runtime/utils/SerdeUtils.java | 123 ---
.../apache/storm/sql/runtime/utils/Utils.java | 55 --
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../apache/storm/sql/TestAvroSerializer.java | 72 --
.../org/apache/storm/sql/TestCsvSerializer.java | 54 --
.../storm/sql/TestJsonRepresentation.java | 52 --
.../org/apache/storm/sql/TestTsvSerializer.java | 46 -
.../test/org/apache/storm/sql/TestUtils.java | 584 -------------
.../socket/TestSocketDataSourceProvider.java | 94 --
external/storm-submit-tools/pom.xml | 216 -----
.../submit/command/DependencyResolverMain.java | 158 ----
.../storm/submit/dependency/AetherUtils.java | 91 --
.../apache/storm/submit/dependency/Booter.java | 51 --
.../submit/dependency/DependencyResolver.java | 98 ---
.../dependency/RepositorySystemFactory.java | 67 --
.../submit/dependency/AetherUtilsTest.java | 102 ---
.../dependency/DependencyResolverTest.java | 83 --
flux/README.md | 872 +++++++++++++++++++
flux/flux-core/pom.xml | 112 +++
.../main/java/org/apache/storm/flux/Flux.java | 259 ++++++
.../java/org/apache/storm/flux/FluxBuilder.java | 630 ++++++++++++++
.../apache/storm/flux/api/TopologySource.java | 39 +
.../org/apache/storm/flux/model/BeanDef.java | 34 +
.../storm/flux/model/BeanListReference.java | 37 +
.../apache/storm/flux/model/BeanReference.java | 39 +
.../org/apache/storm/flux/model/BoltDef.java | 24 +
.../storm/flux/model/ConfigMethodDef.java | 67 ++
.../storm/flux/model/ExecutionContext.java | 77 ++
.../apache/storm/flux/model/GroupingDef.java | 77 ++
.../org/apache/storm/flux/model/IncludeDef.java | 54 ++
.../org/apache/storm/flux/model/ObjectDef.java | 95 ++
.../apache/storm/flux/model/PropertyDef.java | 58 ++
.../org/apache/storm/flux/model/SpoutDef.java | 24 +
.../org/apache/storm/flux/model/StreamDef.java | 64 ++
.../apache/storm/flux/model/TopologyDef.java | 216 +++++
.../storm/flux/model/TopologySourceDef.java | 36 +
.../org/apache/storm/flux/model/VertexDef.java | 36 +
.../apache/storm/flux/parser/FluxParser.java | 206 +++++
flux/flux-core/src/main/resources/splash.txt | 9 +
.../org/apache/storm/flux/FluxBuilderTest.java | 31 +
.../org/apache/storm/flux/IntegrationTest.java | 39 +
.../java/org/apache/storm/flux/TCKTest.java | 254 ++++++
.../multilang/MultilangEnvirontmentTest.java | 89 ++
.../apache/storm/flux/test/SimpleTopology.java | 58 ++
.../storm/flux/test/SimpleTopologySource.java | 52 ++
.../test/SimpleTopologyWithConfigParam.java | 53 ++
.../org/apache/storm/flux/test/TestBolt.java | 131 +++
.../storm/flux/test/TridentTopologySource.java | 71 ++
.../src/test/resources/configs/bad_hbase.yaml | 98 +++
.../test/resources/configs/bad_shell_test.yaml | 118 +++
.../resources/configs/config-methods-test.yaml | 92 ++
.../resources/configs/diamond-topology.yaml | 87 ++
.../existing-topology-method-override.yaml | 25 +
.../existing-topology-reflection-config.yaml | 24 +
.../configs/existing-topology-reflection.yaml | 24 +
.../configs/existing-topology-trident.yaml | 24 +
.../resources/configs/existing-topology.yaml | 23 +
.../src/test/resources/configs/hdfs_test.yaml | 97 +++
.../test/resources/configs/include_test.yaml | 25 +
.../configs/invalid-existing-topology.yaml | 33 +
.../src/test/resources/configs/kafka_test.yaml | 126 +++
.../src/test/resources/configs/shell_test.yaml | 118 +++
.../test/resources/configs/simple_hbase.yaml | 120 +++
.../resources/configs/substitution-test.yaml | 106 +++
.../src/test/resources/configs/tck.yaml | 95 ++
.../src/test/resources/configs/test.properties | 18 +
flux/flux-core/src/test/resources/log4j2.xml | 34 +
flux/flux-examples/README.md | 93 ++
flux/flux-examples/pom.xml | 146 ++++
.../flux/examples/StatefulWordCounter.java | 64 ++
.../storm/flux/examples/TestPrintBolt.java | 39 +
.../storm/flux/examples/TestWindowBolt.java | 47 +
.../storm/flux/examples/WordCountClient.java | 74 ++
.../apache/storm/flux/examples/WordCounter.java | 71 ++
.../src/main/resources/hbase_bolt.properties | 18 +
.../src/main/resources/hdfs_bolt.properties | 26 +
.../src/main/resources/kafka_spout.yaml | 136 +++
.../src/main/resources/multilang.yaml | 89 ++
.../src/main/resources/simple_hbase.yaml | 92 ++
.../src/main/resources/simple_hdfs.yaml | 105 +++
.../resources/simple_stateful_wordcount.yaml | 60 ++
.../src/main/resources/simple_windowing.yaml | 69 ++
.../src/main/resources/simple_wordcount.yaml | 68 ++
flux/flux-ui/README.md | 3 +
flux/flux-wrappers/pom.xml | 51 ++
.../flux/wrappers/bolts/FluxShellBolt.java | 175 ++++
.../storm/flux/wrappers/bolts/LogInfoBolt.java | 44 +
.../flux/wrappers/spouts/FluxShellSpout.java | 175 ++++
.../main/resources/resources/randomsentence.js | 93 ++
.../main/resources/resources/splitsentence.py | 24 +
flux/pom.xml | 64 ++
pom.xml | 6 +-
sql/README.md | 207 +++++
sql/pom.xml | 47 +
sql/storm-sql-core/pom.xml | 279 ++++++
sql/storm-sql-core/src/codegen/config.fmpp | 23 +
sql/storm-sql-core/src/codegen/data/Parser.tdd | 80 ++
.../src/codegen/includes/license.ftl | 17 +
.../src/codegen/includes/parserImpls.ftl | 113 +++
.../storm/sql/AbstractTridentProcessor.java | 56 ++
.../src/jvm/org/apache/storm/sql/StormSql.java | 59 ++
.../jvm/org/apache/storm/sql/StormSqlImpl.java | 290 ++++++
.../org/apache/storm/sql/StormSqlRunner.java | 85 ++
.../sql/calcite/ParallelStreamableTable.java | 35 +
.../apache/storm/sql/compiler/CompilerUtil.java | 183 ++++
.../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 +++++
.../sql/compiler/StormSqlTypeFactoryImpl.java | 51 ++
.../standalone/BuiltinAggregateFunctions.java | 238 +++++
.../backends/standalone/PlanCompiler.java | 139 +++
.../standalone/PostOrderRelNodeVisitor.java | 132 +++
.../backends/standalone/RelNodeCompiler.java | 484 ++++++++++
.../storm/sql/javac/CompilingClassLoader.java | 225 +++++
.../storm/sql/parser/ColumnConstraint.java | 42 +
.../storm/sql/parser/ColumnDefinition.java | 44 +
.../storm/sql/parser/SqlCreateFunction.java | 94 ++
.../apache/storm/sql/parser/SqlCreateTable.java | 166 ++++
.../apache/storm/sql/parser/SqlDDLKeywords.java | 27 +
.../apache/storm/sql/parser/StormParser.java | 47 +
.../apache/storm/sql/parser/UnparseUtil.java | 59 ++
.../sql/planner/StormRelDataTypeSystem.java | 37 +
.../apache/storm/sql/planner/StormRelUtils.java | 68 ++
.../planner/UnsupportedOperatorsVisitor.java | 24 +
.../storm/sql/planner/rel/StormCalcRelBase.java | 31 +
.../sql/planner/rel/StormFilterRelBase.java | 31 +
.../storm/sql/planner/rel/StormJoinRelBase.java | 35 +
.../sql/planner/rel/StormProjectRelBase.java | 34 +
.../storm/sql/planner/rel/StormRelNode.java | 24 +
.../planner/rel/StormStreamInsertRelBase.java | 35 +
.../sql/planner/rel/StormStreamScanRelBase.java | 33 +
.../storm/sql/planner/trident/QueryPlanner.java | 156 ++++
.../sql/planner/trident/TridentPlanCreator.java | 123 +++
.../planner/trident/TridentStormRuleSets.java | 110 +++
.../sql/planner/trident/rel/TridentCalcRel.java | 97 +++
.../planner/trident/rel/TridentFilterRel.java | 65 ++
.../trident/rel/TridentLogicalConvention.java | 67 ++
.../planner/trident/rel/TridentProjectRel.java | 69 ++
.../sql/planner/trident/rel/TridentRel.java | 25 +
.../trident/rel/TridentStreamInsertRel.java | 76 ++
.../trident/rel/TridentStreamScanRel.java | 55 ++
.../trident/rules/TridentAggregateRule.java | 39 +
.../planner/trident/rules/TridentCalcRule.java | 45 +
.../trident/rules/TridentFilterRule.java | 45 +
.../planner/trident/rules/TridentJoinRule.java | 37 +
.../trident/rules/TridentModifyRule.java | 71 ++
.../trident/rules/TridentProjectRule.java | 45 +
.../planner/trident/rules/TridentScanRule.java | 58 ++
.../test/org/apache/storm/sql/TestStormSql.java | 492 +++++++++++
.../storm/sql/compiler/TestExprSemantic.java | 410 +++++++++
.../backends/standalone/TestCompilerUtils.java | 183 ++++
.../backends/standalone/TestPlanCompiler.java | 104 +++
.../standalone/TestRelNodeCompiler.java | 64 ++
.../backends/trident/TestCompilerUtils.java | 208 +++++
.../backends/trident/TestPlanCompiler.java | 232 +++++
.../apache/storm/sql/parser/TestSqlParser.java | 54 ++
sql/storm-sql-external/storm-sql-hdfs/pom.xml | 104 +++
.../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 +++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/hdfs/TestHdfsDataSourcesProvider.java | 129 +++
sql/storm-sql-external/storm-sql-kafka/pom.xml | 93 ++
.../sql/kafka/KafkaDataSourcesProvider.java | 170 ++++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/kafka/TestKafkaDataSourcesProvider.java | 125 +++
.../storm-sql-mongodb/pom.xml | 84 ++
.../sql/mongodb/MongoDataSourcesProvider.java | 126 +++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../mongodb/TestMongoDataSourcesProvider.java | 122 +++
sql/storm-sql-external/storm-sql-redis/pom.xml | 76 ++
.../sql/redis/RedisDataSourcesProvider.java | 228 +++++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/redis/TestRedisDataSourcesProvider.java | 158 ++++
sql/storm-sql-runtime/pom.xml | 136 +++
.../calcite/interpreter/StormContext.java | 31 +
.../sql/runtime/AbstractChannelHandler.java | 52 ++
.../sql/runtime/AbstractValuesProcessor.java | 44 +
.../storm/sql/runtime/ChannelContext.java | 30 +
.../storm/sql/runtime/ChannelHandler.java | 41 +
.../org/apache/storm/sql/runtime/Channels.java | 109 +++
.../apache/storm/sql/runtime/DataSource.java | 27 +
.../storm/sql/runtime/DataSourcesProvider.java | 47 +
.../storm/sql/runtime/DataSourcesRegistry.java | 76 ++
.../org/apache/storm/sql/runtime/FieldInfo.java | 47 +
.../storm/sql/runtime/IOutputSerializer.java | 31 +
.../sql/runtime/ISqlTridentDataSource.java | 65 ++
.../sql/runtime/SimpleSqlTridentConsumer.java | 41 +
.../storm/sql/runtime/StormSqlFunctions.java | 34 +
.../calcite/DebuggableExecutableExpression.java | 45 +
.../runtime/calcite/ExecutableExpression.java | 31 +
.../sql/runtime/calcite/StormDataContext.java | 79 ++
.../socket/SocketDataSourcesProvider.java | 100 +++
.../datasource/socket/trident/SocketState.java | 89 ++
.../socket/trident/SocketStateUpdater.java | 59 ++
.../socket/trident/TridentSocketSpout.java | 177 ++++
.../sql/runtime/serde/avro/AvroScheme.java | 74 ++
.../sql/runtime/serde/avro/AvroSerializer.java | 72 ++
.../sql/runtime/serde/avro/CachedSchemas.java | 41 +
.../storm/sql/runtime/serde/csv/CsvScheme.java | 70 ++
.../sql/runtime/serde/csv/CsvSerializer.java | 59 ++
.../sql/runtime/serde/json/JsonScheme.java | 58 ++
.../sql/runtime/serde/json/JsonSerializer.java | 57 ++
.../storm/sql/runtime/serde/tsv/TsvScheme.java | 58 ++
.../sql/runtime/serde/tsv/TsvSerializer.java | 54 ++
.../trident/functions/EvaluationCalc.java | 86 ++
.../trident/functions/EvaluationFilter.java | 61 ++
.../trident/functions/EvaluationFunction.java | 67 ++
.../trident/functions/ForwardFunction.java | 30 +
.../storm/sql/runtime/utils/FieldInfoUtils.java | 39 +
.../storm/sql/runtime/utils/SerdeUtils.java | 123 +++
.../apache/storm/sql/runtime/utils/Utils.java | 55 ++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../apache/storm/sql/TestAvroSerializer.java | 72 ++
.../org/apache/storm/sql/TestCsvSerializer.java | 54 ++
.../storm/sql/TestJsonRepresentation.java | 52 ++
.../org/apache/storm/sql/TestTsvSerializer.java | 46 +
.../test/org/apache/storm/sql/TestUtils.java | 584 +++++++++++++
.../socket/TestSocketDataSourceProvider.java | 94 ++
.../final-package/src/main/assembly/binary.xml | 87 +-
storm-submit-tools/pom.xml | 216 +++++
.../submit/command/DependencyResolverMain.java | 158 ++++
.../storm/submit/dependency/AetherUtils.java | 91 ++
.../apache/storm/submit/dependency/Booter.java | 51 ++
.../submit/dependency/DependencyResolver.java | 98 +++
.../dependency/RepositorySystemFactory.java | 67 ++
.../submit/dependency/AetherUtilsTest.java | 102 +++
.../dependency/DependencyResolverTest.java | 83 ++
414 files changed, 19519 insertions(+), 19591 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/flux/pom.xml
----------------------------------------------------------------------
diff --cc flux/pom.xml
index 0000000,287b04a..ce8f6a0
mode 000000,100644..100644
--- a/flux/pom.xml
+++ b/flux/pom.xml
@@@ -1,0 -1,85 +1,64 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>flux</artifactId>
+ <packaging>pom</packaging>
+ <name>flux</name>
+
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <developers>
+ <developer>
+ <id>ptgoetz</id>
+ <name>P. Taylor Goetz</name>
+ <email>ptgoetz@apache.org</email>
+ </developer>
+ </developers>
+
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- <!-- see comment below... This fixes an annoyance with intellij -->
- <provided.scope>provided</provided.scope>
- </properties>
-
- <profiles>
- <!--
- Hack to make intellij behave.
- If you use intellij, enable this profile in your IDE.
- It should make life easier.
- -->
- <profile>
- <id>intellij</id>
- <properties>
- <provided.scope>compile</provided.scope>
- </properties>
- </profile>
- </profiles>
-
+ <modules>
+ <module>flux-wrappers</module>
+ <module>flux-core</module>
- <module>flux-examples</module>
+ </modules>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>commons-cli</groupId>
+ <artifactId>commons-cli</artifactId>
+ <version>1.2</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>${storm.kafka.artifact.id}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/pom.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-core/pom.xml
index 0000000,ca84970..e409658
mode 000000,100644..100644
--- a/sql/storm-sql-core/pom.xml
+++ b/sql/storm-sql-core/pom.xml
@@@ -1,0 -1,279 +1,279 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-core</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <version>${calcite.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ <version>${jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-cli</groupId>
+ <artifactId>commons-cli</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ <scope>runtime</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-collections4</artifactId>
+ <version>4.1</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <plugins>
+ <plugin>
+ <artifactId>maven-resources-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-fmpp-resources</id>
+ <phase>initialize</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/codegen</directory>
+ <filtering>false</filtering>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ <execution>
+ <id>copy-java-sources</id>
+ <phase>process-sources</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>${basedir}/target/classes/</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/jvm</directory>
+ <filtering>true</filtering>
+ </resource>
+ <resource>
+ <directory>src/test</directory>
+ <filtering>true</filtering>
+ </resource>
+ <resource>
+ <directory>target/generated-sources</directory>
+ <!-- <include>*/org</include> -->
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.2</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <!-- Extract parser grammar template from calcite-core.jar and put
+ it under ${project.build.directory} where all freemarker templates are. -->
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>2.8</version>
+ <executions>
+ <execution>
+ <id>unpack-parser-template</id>
+ <phase>initialize</phase>
+ <goals>
+ <goal>unpack</goal>
+ </goals>
+ <configuration>
+ <artifactItems>
+ <artifactItem>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <type>jar</type>
+ <overWrite>true</overWrite>
+ <outputDirectory>${project.build.directory}/</outputDirectory>
+ <includes>**/Parser.jj</includes>
+ </artifactItem>
+ </artifactItems>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
+ as copy and unpack goal are not working together -->
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.9</version>
+ <executions>
+ <execution>
+ <id>create-repo</id>
+ <goals>
+ <goal>create-repository</goal>
+ </goals>
+ <configuration>
+ <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+ <repositoryLayout>flat</repositoryLayout>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+ <artifactId>fmpp-maven-plugin</artifactId>
+ <version>1.0</version>
+ <dependencies>
+ <dependency>
+ <groupId>org.freemarker</groupId>
+ <artifactId>freemarker</artifactId>
+ <version>2.3.25-incubating</version>
+ </dependency>
+ </dependencies>
+ <executions>
+ <execution>
+ <id>generate-fmpp-sources</id>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
+ <outputDirectory>target/generated-sources</outputDirectory>
+ <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.5</version>
+ <executions>
+ <execution>
+ <id>add-generated-sources</id>
+ <phase>process-sources</phase>
+ <goals>
+ <goal>add-source</goal>
+ </goals>
+ <configuration>
+ <sources>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <version>2.4</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <id>javacc</id>
+ <goals>
+ <goal>javacc</goal>
+ </goals>
+ <configuration>
+ <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
+ <includes>
+ <include>**/Parser.jj</include>
+ </includes>
+ <lookAhead>2</lookAhead>
+ <isStatic>false</isStatic>
+ <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-hdfs/pom.xml
index 0000000,9275fbb..d1f37b7
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-hdfs/pom.xml
+++ b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
@@@ -1,0 -1,104 +1,104 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-hdfs</artifactId>
+
+ <developers>
+ <developer>
+ <id>vesense</id>
+ <name>Xin Wang</name>
+ <email>data.xinwang@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ <exclusions>
+ <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+ see: http://stackoverflow.com/q/20469026/3542091 -->
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>log4j-over-slf4j</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-hdfs</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>${hadoop.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
+ </exclusions>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-kafka/pom.xml
index 0000000,881343d..c7562bb
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-kafka/pom.xml
+++ b/sql/storm-sql-external/storm-sql-kafka/pom.xml
@@@ -1,0 -1,93 +1,93 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-kafka</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-kafka</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>${storm.kafka.artifact.id}</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>kafka-clients</artifactId>
+ <version>${storm.kafka.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-mongodb/pom.xml
index 0000000,218d89b..cb0baba
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-mongodb/pom.xml
+++ b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
@@@ -1,0 -1,84 +1,84 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-mongodb</artifactId>
+
+ <developers>
+ <developer>
+ <id>vesense</id>
+ <name>Xin Wang</name>
+ <email>data.xinwang@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-mongodb</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-redis/pom.xml
index 0000000,32ce432..38648a4
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-redis/pom.xml
+++ b/sql/storm-sql-external/storm-sql-redis/pom.xml
@@@ -1,0 -1,76 +1,76 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-redis</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-redis</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-runtime/pom.xml
index 0000000,ce57cb2..060a1ab
mode 000000,100644..100644
--- a/sql/storm-sql-runtime/pom.xml
+++ b/sql/storm-sql-runtime/pom.xml
@@@ -1,0 -1,136 +1,136 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+ -->
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-runtime</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
- <scope>provided</scope>
++ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <version>${calcite.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>commons-dbcp</groupId>
+ <artifactId>commons-dbcp</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.pentaho</groupId>
+ <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro</artifactId>
+ <version>1.7.7</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-csv</artifactId>
+ <version>1.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/resources</directory>
+ </resource>
+ </resources>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.9</version>
+ <executions>
+ <execution>
+ <id>create-repo</id>
+ <goals>
+ <goal>create-repository</goal>
+ </goals>
+ <configuration>
+ <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+ <repositoryLayout>flat</repositoryLayout>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/storm-dist/binary/final-package/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --cc storm-dist/binary/final-package/src/main/assembly/binary.xml
index 9321ea2,7ee1445..02b405f
--- a/storm-dist/binary/final-package/src/main/assembly/binary.xml
+++ b/storm-dist/binary/final-package/src/main/assembly/binary.xml
@@@ -180,59 -290,14 +180,7 @@@
<include>README.*</include>
</includes>
</fileSet>
- <!-- Storm SQL -->
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-core/target/app-assembler/repo</directory>
- <outputDirectory>external/sql/storm-sql-core</outputDirectory>
- <includes>
- <include>*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-kafka/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-kafka</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-redis/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-redis</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-mongodb/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-mongodb</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-hdfs/target</directory>
- <outputDirectory>external/sql/storm-sql-external/storm-sql-hdfs</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../../external/sql/storm-sql-runtime/target/app-assembler/repo</directory>
- <outputDirectory>external/sql/storm-sql-runtime</outputDirectory>
- <includes>
- <include>*jar</include>
- </includes>
- </fileSet>
-
- <fileSet>
- <directory>${project.basedir}/../../../external/sql</directory>
- <outputDirectory>external/sql</outputDirectory>
- <includes>
- <include>README.*</include>
- </includes>
- </fileSet>
- <!-- END Storm SQL -->
<fileSet>
- <directory>${project.basedir}/../../../external/storm-cassandra/target</directory>
- <outputDirectory>external/storm-cassandra</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
<directory>${project.basedir}/../../../external/storm-mqtt</directory>
<outputDirectory>external/storm-mqtt</outputDirectory>
<includes>
@@@ -322,6 -451,62 +263,32 @@@
<include>storm*jar</include>
</includes>
</fileSet>
+
- <fileSet>
- <directory>${project.basedir}/../../storm-perf/target</directory>
- <outputDirectory>perf</outputDirectory>
- <includes>
- <include>storm*jar</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../storm-perf/src/main/conf</directory>
- <outputDirectory>perf/conf</outputDirectory>
- <includes>
- <include>*yaml</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../storm-perf/src/main/sampledata</directory>
- <outputDirectory>perf/resources</outputDirectory>
- <includes>
- <include>*</include>
- </includes>
- </fileSet>
- <fileSet>
- <directory>${project.basedir}/../../storm-perf/</directory>
- <outputDirectory>perf</outputDirectory>
- <includes>
- <include>README.*</include>
- </includes>
- </fileSet>
-
+ <!-- storm-sql -->
+ <fileSet>
+ <directory>${project.basedir}/../../../sql/storm-sql-core/target/app-assembler/repo</directory>
+ <outputDirectory>lib-tools/sql/core</outputDirectory>
+ <includes>
+ <include>*jar</include>
+ </includes>
+ </fileSet>
+ <fileSet>
+ <directory>${project.basedir}/../../../sql/storm-sql-runtime/target/app-assembler/repo</directory>
+ <outputDirectory>lib-tools/sql/runtime</outputDirectory>
+ <includes>
+ <include>*jar</include>
+ </includes>
+ </fileSet>
+
+ <!-- storm-submit-tools -->
+ <fileSet>
+ <directory>${project.basedir}/../../../storm-submit-tools/target</directory>
+ <outputDirectory>lib-tools/submit-tools</outputDirectory>
+ <includes>
+ <include>storm*jar</include>
+ </includes>
+ </fileSet>
+
-
</fileSets>
<files>
[07/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
new file mode 100644
index 0000000..05b8e7a
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.wrappers.bolts;
+
+import org.apache.storm.task.ShellBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A generic `ShellBolt` implementation that allows you specify output fields
+ * and even streams without having to subclass `ShellBolt` to do so.
+ *
+ */
+public class FluxShellBolt extends ShellBolt implements IRichBolt{
+ private Map<String, String[]> outputFields;
+ private Map<String, Object> componentConfig;
+
+ /**
+ * Create a ShellBolt with command line arguments
+ * @param command Command line arguments for the bolt
+ */
+ public FluxShellBolt(String[] command){
+ super(command);
+ this.outputFields = new HashMap<String, String[]>();
+ }
+
+ /**
+ * Create a ShellBolt with command line arguments and output fields
+ *
+ * Keep this constructor for backward compatibility.
+ *
+ * @param command Command line arguments for the bolt
+ * @param outputFields Names of fields the bolt will emit (if any).
+ */
+ public FluxShellBolt(String[] command, String[] outputFields){
+ this(command);
+ this.setDefaultStream(outputFields);
+ }
+
+ /**
+ * Add configuration for this bolt. This method is called from YAML file:
+ *
+ * ```
+ * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ * constructorArgs:
+ * # command line
+ * - ["python", "splitsentence.py"]
+ * # output fields
+ * - ["word"]
+ * configMethods:
+ * - name: "addComponentConfig"
+ * args: ["publisher.data_paths", "actions"]
+ * ```
+ *
+ * @param key
+ * @param value
+ */
+ public void addComponentConfig(String key, Object value) {
+ if (this.componentConfig == null) {
+ this.componentConfig = new HashMap<String, Object>();
+ }
+ this.componentConfig.put(key, value);
+ }
+
+ /**
+ * Add configuration for this bolt. This method is called from YAML file:
+ *
+ * ```
+ * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ * constructorArgs:
+ * # command line
+ * - ["python", "splitsentence.py"]
+ * # output fields
+ * - ["word"]
+ * configMethods:
+ * - name: "addComponentConfig"
+ * args:
+ * - "publisher.data_paths"
+ * - ["actions"]
+ * ```
+ *
+ * @param key
+ * @param values
+ */
+ public void addComponentConfig(String key, List<Object> values) {
+ if (this.componentConfig == null) {
+ this.componentConfig = new HashMap<String, Object>();
+ }
+ this.componentConfig.put(key, values);
+ }
+
+ /**
+ * Set default stream outputFields, this method is called from YAML file:
+ *
+ * ```
+ * bolts:
+ * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
+ * id: my_bolt
+ * constructorArgs:
+ * - [python, my_bolt.py]
+ * configMethods:
+ * - name: setDefaultStream
+ * args:
+ * - [word, count]
+ * ```
+ *
+ * @param outputFields Names of fields the bolt will emit (if any) in default stream.
+ */
+ public void setDefaultStream(String[] outputFields) {
+ this.setNamedStream("default", outputFields);
+ }
+
+ /**
+ * Set custom *named* stream outputFields, this method is called from YAML file:
+ *
+ * ```
+ * bolts:
+ * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
+ * id: my_bolt
+ * constructorArgs:
+ * - [python, my_bolt.py]
+ * configMethods:
+ * - name: setNamedStream
+ * args:
+ * - first
+ * - [word, count]
+ * ```
+ * @param name Name of stream the bolt will emit into.
+ * @param outputFields Names of fields the bolt will emit in custom *named* stream.
+ */
+ public void setNamedStream(String name, String[] outputFields) {
+ this.outputFields.put(name, outputFields);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ Iterator it = this.outputFields.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry entryTuple = (Map.Entry)it.next();
+ String key = (String)entryTuple.getKey();
+ String[] value = (String[])entryTuple.getValue();
+ if(key.equals("default")) {
+ declarer.declare(new Fields(value));
+ } else {
+ declarer.declareStream(key, new Fields(value));
+ }
+ }
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return this.componentConfig;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
new file mode 100644
index 0000000..5f0e84b
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.flux.wrappers.bolts;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple bolt that does nothing other than LOG.info() every tuple recieveed.
+ *
+ */
+public class LogInfoBolt extends BaseBasicBolt {
+ private static final Logger LOG = LoggerFactory.getLogger(LogInfoBolt.class);
+
+ @Override
+ public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
+ LOG.info("{}", tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
new file mode 100644
index 0000000..5fd378d
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.wrappers.spouts;
+
+import org.apache.storm.spout.ShellSpout;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+
+/**
+ * A generic `ShellSpout` implementation that allows you specify output fields
+ * and even streams without having to subclass `ShellSpout` to do so.
+ *
+ */
+public class FluxShellSpout extends ShellSpout implements IRichSpout {
+ private Map<String, String[]> outputFields;
+ private Map<String, Object> componentConfig;
+
+ /**
+ * Create a ShellSpout with command line arguments
+ * @param command Command line arguments for the bolt
+ */
+ public FluxShellSpout(String[] command){
+ super(command);
+ this.outputFields = new HashMap<String, String[]>();
+ }
+
+ /**
+ * Create a ShellSpout with command line arguments and output fields
+ *
+ * Keep this constructor for backward compatibility.
+ *
+ * @param args Command line arguments for the spout
+ * @param outputFields Names of fields the spout will emit.
+ */
+ public FluxShellSpout(String[] args, String[] outputFields){
+ this(args);
+ this.setDefaultStream(outputFields);
+ }
+
+ /**
+ * Add configuration for this spout. This method is called from YAML file:
+ *
+ * ```
+ * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
+ * constructorArgs:
+ * # command line
+ * - ["python", "splitsentence.py"]
+ * # output fields
+ * - ["word"]
+ * configMethods:
+ * - name: "addComponentConfig"
+ * args: ["publisher.data_paths", "actions"]
+ * ```
+ *
+ * @param key
+ * @param value
+ */
+ public void addComponentConfig(String key, Object value) {
+ if (this.componentConfig == null) {
+ this.componentConfig = new HashMap<String, Object>();
+ }
+ this.componentConfig.put(key, value);
+ }
+
+ /**
+ * Add configuration for this spout. This method is called from YAML file:
+ *
+ * ```
+ * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
+ * constructorArgs:
+ * # command line
+ * - ["python", "splitsentence.py"]
+ * # output fields
+ * - ["word"]
+ * configMethods:
+ * - name: "addComponentConfig"
+ * args:
+ * - "publisher.data_paths"
+ * - ["actions"]
+ * ```
+ *
+ * @param key
+ * @param values
+ */
+ public void addComponentConfig(String key, List<Object> values) {
+ if (this.componentConfig == null) {
+ this.componentConfig = new HashMap<String, Object>();
+ }
+ this.componentConfig.put(key, values);
+ }
+
+ /**
+ * Set default stream outputFields, this method is called from YAML file:
+ *
+ * ```
+ * spouts:
+ * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
+ * id: my_spout
+ * constructorArgs:
+ * - [python, my_spout.py]
+ * configMethods:
+ * - name: setDefaultStream
+ * args:
+ * - [word, count]
+ * ```
+ *
+ * @param outputFields Names of fields the spout will emit (if any) in default stream.
+ */
+ public void setDefaultStream(String[] outputFields) {
+ this.setNamedStream("default", outputFields);
+ }
+
+ /**
+ * Set custom *named* stream outputFields, this method is called from YAML file:
+ *
+ * ```
+ * spouts:
+ * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
+ * id: my_spout
+ * constructorArgs:
+ * - [python, my_spout.py]
+ * configMethods:
+ * - name: setNamedStream
+ * args:
+ * - first
+ * - [word, count]
+ * ```
+ * @param name Name of stream the spout will emit into.
+ * @param outputFields Names of fields the spout will emit in custom *named* stream.
+ */
+ public void setNamedStream(String name, String[] outputFields) {
+ this.outputFields.put(name, outputFields);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ Iterator it = this.outputFields.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry entryTuple = (Map.Entry)it.next();
+ String key = (String)entryTuple.getKey();
+ String[] value = (String[])entryTuple.getValue();
+ if(key.equals("default")) {
+ declarer.declare(new Fields(value));
+ } else {
+ declarer.declareStream(key, new Fields(value));
+ }
+ }
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ return this.componentConfig;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/resources/resources/randomsentence.js b/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
new file mode 100644
index 0000000..b121915
--- /dev/null
+++ b/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Example for storm spout. Emits random sentences.
+ * The original class in java - org.apache.storm.starter.spout.RandomSentenceSpout.
+ *
+ */
+
+var storm = require('./storm');
+var Spout = storm.Spout;
+
+
+var SENTENCES = [
+ "the cow jumped over the moon",
+ "an apple a day keeps the doctor away",
+ "four score and seven years ago",
+ "snow white and the seven dwarfs",
+ "i am at two with nature"]
+
+function RandomSentenceSpout(sentences) {
+ Spout.call(this);
+ this.runningTupleId = 0;
+ this.sentences = sentences;
+ this.pending = {};
+};
+
+RandomSentenceSpout.prototype = Object.create(Spout.prototype);
+RandomSentenceSpout.prototype.constructor = RandomSentenceSpout;
+
+RandomSentenceSpout.prototype.getRandomSentence = function() {
+ return this.sentences[getRandomInt(0, this.sentences.length - 1)];
+}
+
+RandomSentenceSpout.prototype.nextTuple = function(done) {
+ var self = this;
+ var sentence = this.getRandomSentence();
+ var tup = [sentence];
+ var id = this.createNextTupleId();
+ this.pending[id] = tup;
+ //This timeout can be removed if TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS is configured to 100
+ setTimeout(function() {
+ self.emit({tuple: tup, id: id}, function(taskIds) {
+ self.log(tup + ' sent to task ids - ' + taskIds);
+ });
+ done();
+ },100);
+}
+
+RandomSentenceSpout.prototype.createNextTupleId = function() {
+ var id = this.runningTupleId;
+ this.runningTupleId++;
+ return id;
+}
+
+RandomSentenceSpout.prototype.ack = function(id, done) {
+ this.log('Received ack for - ' + id);
+ delete this.pending[id];
+ done();
+}
+
+RandomSentenceSpout.prototype.fail = function(id, done) {
+ var self = this;
+ this.log('Received fail for - ' + id + '. Retrying.');
+ this.emit({tuple: this.pending[id], id:id}, function(taskIds) {
+ self.log(self.pending[id] + ' sent to task ids - ' + taskIds);
+ });
+ done();
+}
+
+/**
+ * Returns a random integer between min (inclusive) and max (inclusive)
+ */
+function getRandomInt(min, max) {
+ return Math.floor(Math.random() * (max - min + 1)) + min;
+}
+
+new RandomSentenceSpout(SENTENCES).run();
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/resources/resources/splitsentence.py b/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
new file mode 100644
index 0000000..300105f
--- /dev/null
+++ b/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
@@ -0,0 +1,24 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+import storm
+
+class SplitSentenceBolt(storm.BasicBolt):
+ def process(self, tup):
+ words = tup.values[0].split(" ")
+ for word in words:
+ storm.emit([word])
+
+SplitSentenceBolt().run()
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/pom.xml
----------------------------------------------------------------------
diff --git a/flux/pom.xml b/flux/pom.xml
new file mode 100644
index 0000000..287b04a
--- /dev/null
+++ b/flux/pom.xml
@@ -0,0 +1,85 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <artifactId>flux</artifactId>
+ <packaging>pom</packaging>
+ <name>flux</name>
+
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <developers>
+ <developer>
+ <id>ptgoetz</id>
+ <name>P. Taylor Goetz</name>
+ <email>ptgoetz@apache.org</email>
+ </developer>
+ </developers>
+
+ <properties>
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+ <!-- see comment below... This fixes an annoyance with intellij -->
+ <provided.scope>provided</provided.scope>
+ </properties>
+
+ <profiles>
+ <!--
+ Hack to make intellij behave.
+ If you use intellij, enable this profile in your IDE.
+ It should make life easier.
+ -->
+ <profile>
+ <id>intellij</id>
+ <properties>
+ <provided.scope>compile</provided.scope>
+ </properties>
+ </profile>
+ </profiles>
+
+ <modules>
+ <module>flux-wrappers</module>
+ <module>flux-core</module>
+ <module>flux-examples</module>
+ </modules>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>${provided.scope}</scope>
+ </dependency>
+ <dependency>
+ <groupId>commons-cli</groupId>
+ <artifactId>commons-cli</artifactId>
+ <version>1.2</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>${storm.kafka.artifact.id}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 896d735..d3f94e2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -320,6 +320,9 @@
<module>storm-drpc-server</module>
<module>storm-rename-hack</module>
<module>storm-clojure</module>
+ <module>storm-submit-tools</module>
+ <module>flux</module>
+ <module>sql</module>
<!-- externals -->
<module>external/storm-kafka</module>
@@ -329,8 +332,6 @@
<module>external/storm-jdbc</module>
<module>external/storm-redis</module>
<module>external/storm-eventhubs</module>
- <module>external/flux</module>
- <module>external/sql</module>
<module>external/storm-elasticsearch</module>
<module>external/storm-solr</module>
<module>external/storm-metrics</module>
@@ -342,7 +343,6 @@
<module>external/storm-kafka-monitor</module>
<module>external/storm-kinesis</module>
<module>external/storm-druid</module>
- <module>external/storm-submit-tools</module>
<module>external/storm-jms</module>
<module>external/storm-pmml</module>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/README.md
----------------------------------------------------------------------
diff --git a/sql/README.md b/sql/README.md
new file mode 100644
index 0000000..a4b44fb
--- /dev/null
+++ b/sql/README.md
@@ -0,0 +1,207 @@
+# Storm SQL
+
+Compile SQL queries to Storm topologies.
+
+## Usage
+
+Run the ``storm sql`` command to compile SQL statements into Trident topology, and submit it to the Storm cluster
+
+```
+$ bin/storm sql <sql-file> <topo-name>
+```
+
+In which `sql-file` contains a list of SQL statements to be executed, and `topo-name` is the name of the topology.
+
+StormSQL activates `explain mode` and shows query plan instead of submitting topology when user specifies `topo-name` as `--explain`.
+Detailed explanation is available from `Showing Query Plan (explain mode)` section.
+
+## Supported Features
+
+The following features are supported in the current repository:
+
+* Streaming from and to external data sources
+* Filtering tuples
+* Projections
+* Aggregations (Grouping)
+* User defined function (scalar and aggregate)
+* Join (Inner, Left outer, Right outer, Full outer)
+
+## Specifying External Data Sources
+
+In StormSQL data is represented by external tables. Users can specify data sources using the `CREATE EXTERNAL TABLE`
+statement. For example, the following statement specifies a Kafka spouts and sink:
+
+```
+CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY) LOCATION 'kafka://localhost:2181/brokers?topic=test' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+```
+
+The syntax of `CREATE EXTERNAL TABLE` closely follows the one defined in
+[Hive Data Definition Language](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL).
+
+`PARALLELISM` is StormSQL's own keyword which describes parallelism hint for input data source. This is same as providing parallelism hint to Trident Spout.
+Downstream operators are executed with same parallelism before repartition (Aggregation triggers repartition).
+
+Default value is 1, and this option is no effect on output data source. (We might change if needed. Normally repartition is the thing to avoid.)
+
+## Plugging in External Data Sources
+
+Users plug in external data sources through implementing the `ISqlTridentDataSource` interface and registers them using
+the mechanisms of Java's service loader. The external data source will be chosen based on the scheme of the URI of the
+tables. Please refer to the implementation of `storm-sql-kafka` for more details.
+
+## Specifying User Defined Function (UDF)
+
+Users can define user defined function (scalar or aggregate) using `CREATE FUNCTION` statement.
+For example, the following statement defines `MYPLUS` function which uses `org.apache.storm.sql.TestUtils$MyPlus` class.
+
+```
+CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'
+```
+
+Storm SQL determines whether the function as scalar or aggregate by checking which methods are defined.
+If the class defines `evaluate` method, Storm SQL treats the function as `scalar`,
+and if the class defines `add` method, Storm SQL treats the function as `aggregate`.
+
+Example of class for scalar function is here:
+
+```
+ public class MyPlus {
+ public static Integer evaluate(Integer x, Integer y) {
+ return x + y;
+ }
+ }
+
+```
+
+and class for aggregate function is here:
+
+```
+ public class MyConcat {
+ public static String init() {
+ return "";
+ }
+ public static String add(String accumulator, String val) {
+ return accumulator + val;
+ }
+ public static String result(String accumulator) {
+ return accumulator;
+ }
+ }
+```
+
+If users don't define `result` method, result is the last return value of `add` method.
+Users need to define `result` method only when we need to transform accumulated value.
+
+## Example: Filtering Kafka Stream
+
+Let's say there is a Kafka stream that represents the transactions of orders. Each message in the stream contains the id
+of the order, the unit price of the product and the quantity of the orders. The goal is to filter orders where the
+transactions are significant and to insert these orders into another Kafka stream for further analysis.
+
+The user can specify the following SQL statements in the SQL file:
+
+```
+CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+
+CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+
+INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
+```
+
+The first statement defines the table `ORDER` which represents the input stream. The `LOCATION` clause specifies the
+ZkHost (`localhost:2181`), the path of the brokers in ZooKeeper (`/brokers`) and the topic (`orders`).
+The `TBLPROPERTIES` clause specifies the configuration of
+[KafkaProducer](http://kafka.apache.org/documentation.html#newproducerconfigs).
+Current implementation of `storm-sql-kafka` requires specifying both `LOCATION` and `TBLPROPERTIES` clauses even though
+the table is read-only or write-only.
+
+Similarly, the second statement specifies the table `LARGE_ORDERS` which represents the output stream. The third
+statement is a `SELECT` statement which defines the topology: it instructs StormSQL to filter all orders in the external
+table `ORDERS`, calculates the total price and inserts matching records into the Kafka stream specified by
+`LARGE_ORDER`.
+
+To run this example, users need to include the data sources (`storm-sql-kafka` in this case) and its dependency in the
+class path. Dependencies for Storm SQL are automatically handled when users run `storm sql`. Users can include data sources at the submission step like below:
+
+```
+$ bin/storm sql order_filtering.sql order_filtering --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
+```
+
+Above command submits the SQL statements to StormSQL. Users need to modify each artifacts' version if users are using different version of Storm or Kafka.
+
+By now you should be able to see the `order_filtering` topology in the Storm UI.
+
+## Showing Query Plan (explain mode)
+
+Like `explain` on SQL statement, StormSQL provides `explain mode` when running Storm SQL Runner. In explain mode, StormSQL analyzes each query statement (only DML) and show plan instead of submitting topology.
+
+In order to run `explain mode`, you need to provide topology name as `--explain` and run `storm sql` as same as submitting.
+
+For example, when you run the example seen above with explain mode:
+
+```
+$ bin/storm sql order_filtering.sql --explain --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
+```
+
+StormSQL prints out like below:
+
+```
+
+===========================================================
+query>
+CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+-----------------------------------------------------------
+16:53:43.951 [main] INFO o.a.s.s.r.DataSourcesRegistry - Registering scheme kafka with org.apache.storm.sql.kafka.KafkaDataSourcesProvider@4d1bf319
+No plan presented on DDL
+===========================================================
+===========================================================
+query>
+CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+-----------------------------------------------------------
+No plan presented on DDL
+===========================================================
+===========================================================
+query>
+INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
+-----------------------------------------------------------
+plan>
+LogicalTableModify(table=[[LARGE_ORDERS]], operation=[INSERT], updateColumnList=[[]], flattened=[true]), id = 8
+ LogicalProject(ID=[$0], TOTAL=[*($1, $2)]), id = 7
+ LogicalFilter(condition=[>(*($1, $2), 50)]), id = 6
+ EnumerableTableScan(table=[[ORDERS]]), id = 5
+
+===========================================================
+
+```
+
+## Current Limitations
+
+- Windowing is yet to be implemented.
+- Only equi-join (single field equality) is supported for joining table.
+- Joining table only applies within each small batch that comes off of the spout.
+ - Not across batches.
+ - Limitation came from `join` feature of Trident.
+ - Please refer this doc: `Trident API Overview` for details.
+
+## License
+
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements. See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership. The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied. See the License for the
+specific language governing permissions and limitations
+under the License.
+
+## Committer Sponsors
+ * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org))
+ * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/pom.xml
----------------------------------------------------------------------
diff --git a/sql/pom.xml b/sql/pom.xml
new file mode 100644
index 0000000..f085dd8
--- /dev/null
+++ b/sql/pom.xml
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>sql</artifactId>
+ <packaging>pom</packaging>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <modules>
+ <module>storm-sql-core</module>
+ <module>storm-sql-runtime</module>
+ <module>storm-sql-external/storm-sql-kafka</module>
+ <module>storm-sql-external/storm-sql-redis</module>
+ <module>storm-sql-external/storm-sql-mongodb</module>
+ <module>storm-sql-external/storm-sql-hdfs</module>
+ </modules>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/pom.xml b/sql/storm-sql-core/pom.xml
new file mode 100644
index 0000000..ca84970
--- /dev/null
+++ b/sql/storm-sql-core/pom.xml
@@ -0,0 +1,279 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <artifactId>storm</artifactId>
+ <groupId>org.apache.storm</groupId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>storm-sql-core</artifactId>
+
+ <developers>
+ <developer>
+ <id>haohui</id>
+ <name>Haohui Mai</name>
+ <email>ricetons@gmail.com</email>
+ </developer>
+ </developers>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-core</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.storm</groupId>
+ <artifactId>storm-sql-runtime</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <version>${calcite.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ <version>${jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-cli</groupId>
+ <artifactId>commons-cli</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ <scope>runtime</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-collections4</artifactId>
+ <version>4.1</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <sourceDirectory>src/jvm</sourceDirectory>
+ <testSourceDirectory>src/test</testSourceDirectory>
+ <plugins>
+ <plugin>
+ <artifactId>maven-resources-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-fmpp-resources</id>
+ <phase>initialize</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/codegen</directory>
+ <filtering>false</filtering>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ <execution>
+ <id>copy-java-sources</id>
+ <phase>process-sources</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>${basedir}/target/classes/</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/jvm</directory>
+ <filtering>true</filtering>
+ </resource>
+ <resource>
+ <directory>src/test</directory>
+ <filtering>true</filtering>
+ </resource>
+ <resource>
+ <directory>target/generated-sources</directory>
+ <!-- <include>*/org</include> -->
+ <filtering>true</filtering>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.2</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <!-- Extract parser grammar template from calcite-core.jar and put
+ it under ${project.build.directory} where all freemarker templates are. -->
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <version>2.8</version>
+ <executions>
+ <execution>
+ <id>unpack-parser-template</id>
+ <phase>initialize</phase>
+ <goals>
+ <goal>unpack</goal>
+ </goals>
+ <configuration>
+ <artifactItems>
+ <artifactItem>
+ <groupId>org.apache.calcite</groupId>
+ <artifactId>calcite-core</artifactId>
+ <type>jar</type>
+ <overWrite>true</overWrite>
+ <outputDirectory>${project.build.directory}/</outputDirectory>
+ <includes>**/Parser.jj</includes>
+ </artifactItem>
+ </artifactItems>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
+ as copy and unpack goal are not working together -->
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.9</version>
+ <executions>
+ <execution>
+ <id>create-repo</id>
+ <goals>
+ <goal>create-repository</goal>
+ </goals>
+ <configuration>
+ <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+ <repositoryLayout>flat</repositoryLayout>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+ <artifactId>fmpp-maven-plugin</artifactId>
+ <version>1.0</version>
+ <dependencies>
+ <dependency>
+ <groupId>org.freemarker</groupId>
+ <artifactId>freemarker</artifactId>
+ <version>2.3.25-incubating</version>
+ </dependency>
+ </dependencies>
+ <executions>
+ <execution>
+ <id>generate-fmpp-sources</id>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ <configuration>
+ <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
+ <outputDirectory>target/generated-sources</outputDirectory>
+ <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.5</version>
+ <executions>
+ <execution>
+ <id>add-generated-sources</id>
+ <phase>process-sources</phase>
+ <goals>
+ <goal>add-source</goal>
+ </goals>
+ <configuration>
+ <sources>
+ <source>${project.build.directory}/generated-sources</source>
+ </sources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>javacc-maven-plugin</artifactId>
+ <version>2.4</version>
+ <executions>
+ <execution>
+ <phase>generate-sources</phase>
+ <id>javacc</id>
+ <goals>
+ <goal>javacc</goal>
+ </goals>
+ <configuration>
+ <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
+ <includes>
+ <include>**/Parser.jj</include>
+ </includes>
+ <lookAhead>2</lookAhead>
+ <isStatic>false</isStatic>
+ <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/config.fmpp b/sql/storm-sql-core/src/codegen/config.fmpp
new file mode 100644
index 0000000..be5a792
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/config.fmpp
@@ -0,0 +1,23 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http:# www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+data: {
+ parser: tdd(../data/Parser.tdd)
+}
+
+freemarkerLinks: {
+ includes: includes/
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/data/Parser.tdd b/sql/storm-sql-core/src/codegen/data/Parser.tdd
new file mode 100644
index 0000000..b0dccb6
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/data/Parser.tdd
@@ -0,0 +1,80 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http:# www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+{
+ # Generated parser implementation class package and name
+ package: "org.apache.storm.sql.parser.impl",
+ class: "StormParserImpl",
+
+ # List of import statements.
+ imports: [
+ "org.apache.calcite.sql.validate.*",
+ "org.apache.calcite.util.*",
+ "org.apache.storm.sql.parser.*",
+ "java.util.*"
+ ]
+
+ # List of keywords.
+ keywords: [
+ "LOCATION",
+ "INPUTFORMAT",
+ "OUTPUTFORMAT",
+ "PARALLELISM",
+ "STORED",
+ "TBLPROPERTIES",
+ "JAR"
+ ]
+
+ # List of methods for parsing custom SQL statements.
+ statementParserMethods: [
+ "SqlCreateTable()",
+ "SqlCreateFunction()"
+ ]
+
+ # List of methods for parsing custom literals.
+ # Example: ParseJsonLiteral().
+ literalParserMethods: [
+ ]
+
+ # List of methods for parsing custom data types.
+ dataTypeParserMethods: [
+ ]
+
+ nonReservedKeywords: [
+ ]
+
+ createStatementParserMethods: [
+ ]
+
+ alterStatementParserMethods: [
+ ]
+
+ dropStatementParserMethods: [
+ ]
+
+ # List of files in @includes directory that have parser method
+ # implementations for custom SQL statements, literals or types
+ # given as part of "statementParserMethods", "literalParserMethods" or
+ # "dataTypeParserMethods".
+ implementationFiles: [
+ "parserImpls.ftl"
+ ]
+
+ includeCompoundIdentifier: true,
+ includeBraces: true,
+ includeAdditionalDeclarations: false,
+ allowBangEqual: false
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/includes/license.ftl
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/includes/license.ftl b/sql/storm-sql-core/src/codegen/includes/license.ftl
new file mode 100644
index 0000000..7e66353
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/includes/license.ftl
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl b/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
new file mode 100644
index 0000000..4143840
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
@@ -0,0 +1,113 @@
+<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+ license agreements. See the NOTICE file distributed with this work for additional
+ information regarding copyright ownership. The ASF licenses this file to
+ You under the Apache License, Version 2.0 (the "License"); you may not use
+ this file except in compliance with the License. You may obtain a copy of
+ the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+ by applicable law or agreed to in writing, software distributed under the
+ License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+ OF ANY KIND, either express or implied. See the License for the specific
+ language governing permissions and limitations under the License. -->
+
+
+private void ColumnDef(List<ColumnDefinition> list) :
+{
+ SqlParserPos pos;
+ SqlIdentifier name;
+ SqlDataTypeSpec type;
+ ColumnConstraint constraint = null;
+ SqlMonotonicity monotonicity = SqlMonotonicity.NOT_MONOTONIC;
+}
+{
+ name = SimpleIdentifier() { pos = getPos(); }
+ type = DataType()
+ [
+ <PRIMARY> <KEY>
+ [ <ASC> { monotonicity = SqlMonotonicity.INCREASING; }
+ | <DESC> { monotonicity = SqlMonotonicity.DECREASING; }
+ ]
+ { constraint = new ColumnConstraint.PrimaryKey(monotonicity, getPos()); }
+ ]
+ {
+ list.add(new ColumnDefinition(name, type, constraint, pos));
+ }
+}
+
+SqlNodeList ColumnDefinitionList() :
+{
+ SqlParserPos pos;
+ List<ColumnDefinition> list = Lists.newArrayList();
+}
+{
+ <LPAREN> { pos = getPos(); }
+ ColumnDef(list)
+ ( <COMMA> ColumnDef(list) )*
+ <RPAREN> {
+ return new SqlNodeList(list, pos.plus(getPos()));
+ }
+}
+
+/**
+ * CREATE EXTERNAL TABLE ( IF NOT EXISTS )?
+ * ( database_name '.' )? table_name ( '(' column_def ( ',' column_def )* ')'
+ * ( STORED AS INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname )?
+ * LOCATION location_uri
+ * ( TBLPROPERTIES tbl_properties )?
+ * ( AS select_stmt )
+ */
+SqlNode SqlCreateTable() :
+{
+ SqlParserPos pos;
+ SqlIdentifier tblName;
+ SqlNodeList fieldList;
+ SqlNode location;
+ SqlNode parallelism = null;
+ SqlNode input_format_class_name = null, output_format_class_name = null;
+ SqlNode tbl_properties = null;
+ SqlNode select = null;
+}
+{
+ <CREATE> { pos = getPos(); }
+ <EXTERNAL> <TABLE>
+ tblName = CompoundIdentifier()
+ fieldList = ColumnDefinitionList()
+ [
+ <STORED> <AS>
+ <INPUTFORMAT> input_format_class_name = StringLiteral()
+ <OUTPUTFORMAT> output_format_class_name = StringLiteral()
+ ]
+ <LOCATION>
+ location = StringLiteral()
+ [ <PARALLELISM> parallelism = UnsignedNumericLiteral() ]
+ [ <TBLPROPERTIES> tbl_properties = StringLiteral() ]
+ [ <AS> select = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) ] {
+ return new SqlCreateTable(pos, tblName, fieldList,
+ input_format_class_name, output_format_class_name, location,
+ parallelism, tbl_properties, select);
+ }
+}
+
+/**
+ * CREATE FUNCTION functionname AS 'classname'
+ */
+SqlNode SqlCreateFunction() :
+{
+ SqlParserPos pos;
+ SqlIdentifier functionName;
+ SqlNode className;
+ SqlNode jarName = null;
+}
+{
+ <CREATE> { pos = getPos(); }
+ <FUNCTION>
+ functionName = CompoundIdentifier()
+ <AS>
+ className = StringLiteral()
+ [
+ <USING> <JAR>
+ jarName = StringLiteral()
+ ]
+ {
+ return new SqlCreateFunction(pos, functionName, className, jarName);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
new file mode 100644
index 0000000..843339f
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import org.apache.calcite.DataContext;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.TridentTopology;
+
+import java.util.List;
+
+public abstract class AbstractTridentProcessor {
+ protected Stream outputStream;
+ protected DataContext dataContext;
+ protected List<CompilingClassLoader> classLoaders;
+ /**
+ * @return the output stream of the SQL
+ */
+ public Stream outputStream() {
+ return outputStream;
+ }
+
+ /**
+ * Construct the trident topology based on the SQL.
+ */
+ public abstract TridentTopology build();
+
+ /**
+ * @return DataContext instance which is used with execution of query
+ */
+ public DataContext getDataContext() {
+ return dataContext;
+ }
+
+ /**
+ * @return Classloaders to compile. They're all chaining so the last classloader can access all classes.
+ */
+ public List<CompilingClassLoader> getClassLoaders() {
+ return classLoaders;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
new file mode 100644
index 0000000..5dec4af
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.sql.runtime.ChannelHandler;
+
+import java.util.Map;
+
+/**
+ * The StormSql class provides standalone, interactive interfaces to execute
+ * SQL statements over streaming data.
+ * <p>
+ * The StormSql class is stateless. The user needs to submit the data
+ * definition language (DDL) statements and the query statements in the same
+ * batch.
+ */
+public abstract class StormSql {
+ /**
+ * Execute the SQL statements in stand-alone mode. The user can retrieve the result by passing in an instance
+ * of {@see ChannelHandler}.
+ */
+ public abstract void execute(Iterable<String> statements,
+ ChannelHandler handler) throws Exception;
+
+ /**
+ * Submit the SQL statements to Nimbus and run it as a topology.
+ */
+ public abstract void submit(
+ String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
+ StormSubmitter.ProgressListener progressListener, String asUser)
+ throws Exception;
+
+ /**
+ * Print out query plan for each query.
+ */
+ public abstract void explain(Iterable<String> statements) throws Exception;
+
+ public static StormSql construct() {
+ return new StormSqlImpl();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
new file mode 100644
index 0000000..007daa7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
@@ -0,0 +1,290 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.sql.compiler.StormSqlTypeFactoryImpl;
+import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.parser.ColumnConstraint;
+import org.apache.storm.sql.parser.ColumnDefinition;
+import org.apache.storm.sql.parser.SqlCreateFunction;
+import org.apache.storm.sql.parser.SqlCreateTable;
+import org.apache.storm.sql.parser.StormParser;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.TridentTopology;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.jar.Attributes;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+import java.util.zip.ZipEntry;
+
+import static org.apache.storm.sql.compiler.CompilerUtil.TableBuilderInfo;
+
+class StormSqlImpl extends StormSql {
+ private final JavaTypeFactory typeFactory = new StormSqlTypeFactoryImpl(
+ RelDataTypeSystem.DEFAULT);
+ private final SchemaPlus schema = Frameworks.createRootSchema(true);
+ private boolean hasUdf = false;
+
+ @Override
+ public void execute(
+ Iterable<String> statements, ChannelHandler result)
+ throws Exception {
+ Map<String, DataSource> dataSources = new HashMap<>();
+ for (String sql : statements) {
+ StormParser parser = new StormParser(sql);
+ SqlNode node = parser.impl().parseSqlStmtEof();
+ if (node instanceof SqlCreateTable) {
+ handleCreateTable((SqlCreateTable) node, dataSources);
+ } else if (node instanceof SqlCreateFunction) {
+ handleCreateFunction((SqlCreateFunction) node);
+ } else {
+ FrameworkConfig config = buildFrameWorkConfig();
+ Planner planner = Frameworks.getPlanner(config);
+ SqlNode parse = planner.parse(sql);
+ SqlNode validate = planner.validate(parse);
+ RelNode tree = planner.convert(validate);
+ PlanCompiler compiler = new PlanCompiler(typeFactory);
+ AbstractValuesProcessor proc = compiler.compile(tree);
+ proc.initialize(dataSources, result);
+ }
+ }
+ }
+
+ @Override
+ public void submit(
+ String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
+ StormSubmitter.ProgressListener progressListener, String asUser)
+ throws Exception {
+ Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
+ for (String sql : statements) {
+ StormParser parser = new StormParser(sql);
+ SqlNode node = parser.impl().parseSqlStmtEof();
+ if (node instanceof SqlCreateTable) {
+ handleCreateTableForTrident((SqlCreateTable) node, dataSources);
+ } else if (node instanceof SqlCreateFunction) {
+ handleCreateFunction((SqlCreateFunction) node);
+ } else {
+ QueryPlanner planner = new QueryPlanner(schema);
+ AbstractTridentProcessor processor = planner.compile(dataSources, sql);
+ TridentTopology topo = processor.build();
+
+ Path jarPath = null;
+ try {
+ // QueryPlanner on Trident mode configures the topology with compiled classes,
+ // so we need to add new classes into topology jar
+ // Topology will be serialized and sent to Nimbus, and deserialized and executed in workers.
+
+ jarPath = Files.createTempFile("storm-sql", ".jar");
+ System.setProperty("storm.jar", jarPath.toString());
+ packageTopology(jarPath, processor);
+ StormSubmitter.submitTopologyAs(name, stormConf, topo.build(), opts, progressListener, asUser);
+ } finally {
+ if (jarPath != null) {
+ Files.delete(jarPath);
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public void explain(Iterable<String> statements) throws Exception {
+ Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
+ for (String sql : statements) {
+ StormParser parser = new StormParser(sql);
+ SqlNode node = parser.impl().parseSqlStmtEof();
+
+ System.out.println("===========================================================");
+ System.out.println("query>");
+ System.out.println(sql);
+ System.out.println("-----------------------------------------------------------");
+
+ if (node instanceof SqlCreateTable) {
+ handleCreateTableForTrident((SqlCreateTable) node, dataSources);
+ System.out.println("No plan presented on DDL");
+ } else if (node instanceof SqlCreateFunction) {
+ handleCreateFunction((SqlCreateFunction) node);
+ System.out.println("No plan presented on DDL");
+ } else {
+ FrameworkConfig config = buildFrameWorkConfig();
+ Planner planner = Frameworks.getPlanner(config);
+ SqlNode parse = planner.parse(sql);
+ SqlNode validate = planner.validate(parse);
+ RelNode tree = planner.convert(validate);
+
+ String plan = StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES);
+ System.out.println("plan>");
+ System.out.println(plan);
+ }
+
+ System.out.println("===========================================================");
+ }
+ }
+
+ private void packageTopology(Path jar, AbstractTridentProcessor processor) throws IOException {
+ Manifest manifest = new Manifest();
+ Attributes attr = manifest.getMainAttributes();
+ attr.put(Attributes.Name.MANIFEST_VERSION, "1.0");
+ attr.put(Attributes.Name.MAIN_CLASS, processor.getClass().getCanonicalName());
+ try (JarOutputStream out = new JarOutputStream(
+ new BufferedOutputStream(new FileOutputStream(jar.toFile())), manifest)) {
+ List<CompilingClassLoader> classLoaders = processor.getClassLoaders();
+ if (classLoaders != null && !classLoaders.isEmpty()) {
+ for (CompilingClassLoader classLoader : classLoaders) {
+ for (Map.Entry<String, ByteArrayOutputStream> e : classLoader.getClasses().entrySet()) {
+ out.putNextEntry(new ZipEntry(e.getKey().replace(".", "/") + ".class"));
+ out.write(e.getValue().toByteArray());
+ out.closeEntry();
+ }
+ }
+ }
+ }
+ }
+
+ private void handleCreateTable(
+ SqlCreateTable n, Map<String, DataSource> dataSources) {
+ List<FieldInfo> fields = updateSchema(n);
+ DataSource ds = DataSourcesRegistry.construct(n.location(), n
+ .inputFormatClass(), n.outputFormatClass(), fields);
+ if (ds == null) {
+ throw new RuntimeException("Cannot construct data source for " + n
+ .tableName());
+ } else if (dataSources.containsKey(n.tableName())) {
+ throw new RuntimeException("Duplicated definition for table " + n
+ .tableName());
+ }
+ dataSources.put(n.tableName(), ds);
+ }
+
+ private void handleCreateFunction(SqlCreateFunction sqlCreateFunction) throws ClassNotFoundException {
+ if(sqlCreateFunction.jarName() != null) {
+ throw new UnsupportedOperationException("UDF 'USING JAR' not implemented");
+ }
+ Method method;
+ Function function;
+ if ((method=findMethod(sqlCreateFunction.className(), "evaluate")) != null) {
+ function = ScalarFunctionImpl.create(method);
+ } else if (findMethod(sqlCreateFunction.className(), "add") != null) {
+ function = AggregateFunctionImpl.create(Class.forName(sqlCreateFunction.className()));
+ } else {
+ throw new RuntimeException("Invalid scalar or aggregate function");
+ }
+ schema.add(sqlCreateFunction.functionName().toUpperCase(), function);
+ hasUdf = true;
+ }
+
+ private Method findMethod(String clazzName, String methodName) throws ClassNotFoundException {
+ Class<?> clazz = Class.forName(clazzName);
+ for (Method method : clazz.getMethods()) {
+ if (method.getName().equals(methodName)) {
+ return method;
+ }
+ }
+ return null;
+ }
+
+ private void handleCreateTableForTrident(
+ SqlCreateTable n, Map<String, ISqlTridentDataSource> dataSources) {
+ List<FieldInfo> fields = updateSchema(n);
+ ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(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);
+ }
+
+ private List<FieldInfo> updateSchema(SqlCreateTable n) {
+ TableBuilderInfo builder = new 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);
+ return fields;
+ }
+
+ private FrameworkConfig buildFrameWorkConfig() {
+ if (hasUdf) {
+ List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+ sqlOperatorTables.add(SqlStdOperatorTable.instance());
+ sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+ false,
+ Collections.<String>emptyList(), typeFactory));
+ return Frameworks.newConfigBuilder().defaultSchema(schema)
+ .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables)).build();
+ } else {
+ return Frameworks.newConfigBuilder().defaultSchema(schema).build();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
new file mode 100644
index 0000000..5618647
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.generated.TopologyInitialStatus;
+import org.apache.storm.utils.Utils;
+
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
+public class StormSqlRunner {
+ private static final String OPTION_SQL_FILE_SHORT = "f";
+ private static final String OPTION_SQL_FILE_LONG = "file";
+ private static final String OPTION_SQL_TOPOLOGY_NAME_SHORT = "t";
+ private static final String OPTION_SQL_TOPOLOGY_NAME_LONG = "topology";
+ private static final String OPTION_SQL_EXPLAIN_SHORT = "e";
+ private static final String OPTION_SQL_EXPLAIN_LONG = "explain";
+
+ public static void main(String[] args) throws Exception {
+ Options options = buildOptions();
+ CommandLineParser parser = new DefaultParser();
+ CommandLine commandLine = parser.parse(options, args);
+
+ if (!commandLine.hasOption(OPTION_SQL_FILE_LONG)) {
+ printUsageAndExit(options, OPTION_SQL_FILE_LONG + " is required");
+ }
+
+ String filePath = commandLine.getOptionValue(OPTION_SQL_FILE_LONG);
+ List<String> stmts = Files.readAllLines(Paths.get(filePath), StandardCharsets.UTF_8);
+ StormSql sql = StormSql.construct();
+ @SuppressWarnings("unchecked")
+ Map<String, ?> conf = Utils.readStormConfig();
+
+ if (commandLine.hasOption(OPTION_SQL_EXPLAIN_LONG)) {
+ sql.explain(stmts);
+ } else if (commandLine.hasOption(OPTION_SQL_TOPOLOGY_NAME_LONG)) {
+ String topoName = commandLine.getOptionValue(OPTION_SQL_TOPOLOGY_NAME_LONG);
+ SubmitOptions submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+ sql.submit(topoName, stmts, conf, submitOptions, null, null);
+ } else {
+ printUsageAndExit(options, "Either " + OPTION_SQL_TOPOLOGY_NAME_LONG + " or " + OPTION_SQL_EXPLAIN_LONG +
+ " must be presented");
+ }
+ }
+
+ private static void printUsageAndExit(Options options, String message) {
+ System.out.println(message);
+ HelpFormatter formatter = new HelpFormatter();
+ formatter.printHelp("storm-sql-runner ", options);
+ System.exit(1);
+ }
+
+ private static Options buildOptions() {
+ Options options = new Options();
+ options.addOption(OPTION_SQL_FILE_SHORT, OPTION_SQL_FILE_LONG, true, "REQUIRED SQL file which has sql statements");
+ options.addOption(OPTION_SQL_TOPOLOGY_NAME_SHORT, OPTION_SQL_TOPOLOGY_NAME_LONG, true, "Topology name to submit");
+ options.addOption(OPTION_SQL_EXPLAIN_SHORT, OPTION_SQL_EXPLAIN_LONG, false, "Activate explain mode (topology name will be ignored)");
+ return options;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
new file mode 100644
index 0000000..c6b584d
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.calcite;
+
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.calcite.schema.StreamableTable;
+
+/**
+ * Table that can be converted to a stream. This table also has its parallelism information.
+ *
+ * @see Delta
+ */
+public interface ParallelStreamableTable extends StreamableTable {
+
+ /**
+ * Returns parallelism hint of this table. Returns null if don't know.
+ */
+ Integer parallelismHint();
+}
[09/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
new file mode 100644
index 0000000..1520006
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import org.apache.storm.Config;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.topology.IRichSpout;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Container for all the objects required to instantiate a topology.
+ */
+public class ExecutionContext {
+ // parsed Topology definition
+ TopologyDef topologyDef;
+
+ // Storm config
+ private Config config;
+
+ // components
+ private List<Object> compontents;
+ // indexed by id
+ private Map<String, Object> componentMap = new HashMap<String, Object>();
+
+ private Map<String, IRichSpout> spoutMap = new HashMap<String, IRichSpout>();
+
+ private List<IBolt> bolts;
+ private Map<String, Object> boltMap = new HashMap<String, Object>();
+
+ public ExecutionContext(TopologyDef topologyDef, Config config){
+ this.topologyDef = topologyDef;
+ this.config = config;
+ }
+
+ public TopologyDef getTopologyDef(){
+ return this.topologyDef;
+ }
+
+ public void addSpout(String id, IRichSpout spout){
+ this.spoutMap.put(id, spout);
+ }
+
+ public void addBolt(String id, Object bolt){
+ this.boltMap.put(id, bolt);
+ }
+
+ public Object getBolt(String id){
+ return this.boltMap.get(id);
+ }
+
+ public void addComponent(String id, Object value){
+ this.componentMap.put(id, value);
+ }
+
+ public Object getComponent(String id){
+ return this.componentMap.get(id);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
new file mode 100644
index 0000000..e4fac8e
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import java.util.List;
+
+/**
+ * Bean representation of a Storm stream grouping.
+ */
+public class GroupingDef {
+
+ /**
+ * Types of stream groupings Storm allows
+ */
+ public static enum Type {
+ ALL,
+ CUSTOM,
+ DIRECT,
+ SHUFFLE,
+ LOCAL_OR_SHUFFLE,
+ FIELDS,
+ GLOBAL,
+ NONE
+ }
+
+ private Type type;
+ private String streamId;
+ private List<String> args;
+ private ObjectDef customClass;
+
+ public List<String> getArgs() {
+ return args;
+ }
+
+ public void setArgs(List<String> args) {
+ this.args = args;
+ }
+
+ public Type getType() {
+ return type;
+ }
+
+ public void setType(Type type) {
+ this.type = type;
+ }
+
+ public String getStreamId() {
+ return streamId;
+ }
+
+ public void setStreamId(String streamId) {
+ this.streamId = streamId;
+ }
+
+ public ObjectDef getCustomClass() {
+ return customClass;
+ }
+
+ public void setCustomClass(ObjectDef customClass) {
+ this.customClass = customClass;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
new file mode 100644
index 0000000..23fd9d2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * Represents an include. Includes can be either a file or a classpath resource.
+ *
+ * If an include is marked as `override=true` then existing properties will be replaced.
+ *
+ */
+public class IncludeDef {
+ private boolean resource = false;
+ boolean override = false;
+ private String file;
+
+ public boolean isResource() {
+ return resource;
+ }
+
+ public void setResource(boolean resource) {
+ this.resource = resource;
+ }
+
+ public String getFile() {
+ return file;
+ }
+
+ public void setFile(String file) {
+ this.file = file;
+ }
+
+ public boolean isOverride() {
+ return override;
+ }
+
+ public void setOverride(boolean override) {
+ this.override = override;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
new file mode 100644
index 0000000..04a7e8a
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import org.apache.storm.Config;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A representation of a Java object that given a className, constructor arguments,
+ * and properties, can be instantiated.
+ */
+public class ObjectDef {
+ private String className;
+ private List<Object> constructorArgs;
+ private boolean hasReferences;
+ private List<PropertyDef> properties;
+ private List<ConfigMethodDef> configMethods;
+
+ public String getClassName() {
+ return className;
+ }
+
+ public void setClassName(String className) {
+ this.className = className;
+ }
+
+ public List<Object> getConstructorArgs() {
+ return constructorArgs;
+ }
+
+ public void setConstructorArgs(List<Object> constructorArgs) {
+
+ List<Object> newVal = new ArrayList<Object>();
+ for(Object obj : constructorArgs){
+ if(obj instanceof LinkedHashMap){
+ Map map = (Map)obj;
+ if(map.containsKey("ref") && map.size() == 1) {
+ newVal.add(new BeanReference((String) map.get("ref")));
+ this.hasReferences = true;
+ } else if (map.containsKey("reflist") && map.size() == 1) {
+ newVal.add(new BeanListReference((List<String>) map.get("reflist")));
+ this.hasReferences = true;
+ } else {
+ newVal.add(obj);
+ }
+ } else {
+ newVal.add(obj);
+ }
+ }
+ this.constructorArgs = newVal;
+ }
+
+ public boolean hasConstructorArgs(){
+ return this.constructorArgs != null && this.constructorArgs.size() > 0;
+ }
+
+ public boolean hasReferences(){
+ return this.hasReferences;
+ }
+
+ public List<PropertyDef> getProperties() {
+ return properties;
+ }
+
+ public void setProperties(List<PropertyDef> properties) {
+ this.properties = properties;
+ }
+
+ public List<ConfigMethodDef> getConfigMethods() {
+ return configMethods;
+ }
+
+ public void setConfigMethods(List<ConfigMethodDef> configMethods) {
+ this.configMethods = configMethods;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
new file mode 100644
index 0000000..f3d7704
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+public class PropertyDef {
+ private String name;
+ private Object value;
+ private String ref;
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public Object getValue() {
+ return value;
+ }
+
+ public void setValue(Object value) {
+ if(this.ref != null){
+ throw new IllegalStateException("A property can only have a value OR a reference, not both.");
+ }
+ this.value = value;
+ }
+
+ public String getRef() {
+ return ref;
+ }
+
+ public void setRef(String ref) {
+ if(this.value != null){
+ throw new IllegalStateException("A property can only have a value OR a reference, not both.");
+ }
+ this.ref = ref;
+ }
+
+ public boolean isReference(){
+ return this.ref != null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
new file mode 100644
index 0000000..277c601
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * Bean representation of a Storm spout.
+ */
+public class SpoutDef extends VertexDef {
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
new file mode 100644
index 0000000..da80f1c
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * Represents a stream of tuples from one Storm component (Spout or Bolt) to another (an edge in the topology DAG).
+ *
+ * Required fields are `from` and `to`, which define the source and destination, and the stream `grouping`.
+ *
+ */
+public class StreamDef {
+
+ private String name; // not used, placeholder for GUI, etc.
+ private String from;
+ private String to;
+ private GroupingDef grouping;
+
+ public String getTo() {
+ return to;
+ }
+
+ public void setTo(String to) {
+ this.to = to;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public String getFrom() {
+ return from;
+ }
+
+ public void setFrom(String from) {
+ this.from = from;
+ }
+
+ public GroupingDef getGrouping() {
+ return grouping;
+ }
+
+ public void setGrouping(GroupingDef grouping) {
+ this.grouping = grouping;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
new file mode 100644
index 0000000..86614f1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * Bean represenation of a topology.
+ *
+ * It consists of the following:
+ * 1. The topology name
+ * 2. A `java.util.Map` representing the `org.apache.storm.config` for the topology
+ * 3. A list of spout definitions
+ * 4. A list of bolt definitions
+ * 5. A list of stream definitions that define the flow between spouts and bolts.
+ *
+ */
+public class TopologyDef {
+ private static Logger LOG = LoggerFactory.getLogger(TopologyDef.class);
+
+ private String name;
+ private Map<String, BeanDef> componentMap = new LinkedHashMap<String, BeanDef>(); // not required
+ private List<IncludeDef> includes; // not required
+ private Map<String, Object> config = new HashMap<String, Object>();
+
+ // a "topology source" is a class that can produce a `StormTopology` thrift object.
+ private TopologySourceDef topologySource;
+
+ // the following are required if we're defining a core storm topology DAG in YAML, etc.
+ private Map<String, BoltDef> boltMap = new LinkedHashMap<String, BoltDef>();
+ private Map<String, SpoutDef> spoutMap = new LinkedHashMap<String, SpoutDef>();
+ private List<StreamDef> streams = new ArrayList<StreamDef>();
+
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public void setName(String name, boolean override){
+ if(this.name == null || override){
+ this.name = name;
+ } else {
+ LOG.warn("Ignoring attempt to set property 'name' with override == false.");
+ }
+ }
+
+ public List<SpoutDef> getSpouts() {
+ ArrayList<SpoutDef> retval = new ArrayList<SpoutDef>();
+ retval.addAll(this.spoutMap.values());
+ return retval;
+ }
+
+ public void setSpouts(List<SpoutDef> spouts) {
+ this.spoutMap = new LinkedHashMap<String, SpoutDef>();
+ for(SpoutDef spout : spouts){
+ this.spoutMap.put(spout.getId(), spout);
+ }
+ }
+
+ public List<BoltDef> getBolts() {
+ ArrayList<BoltDef> retval = new ArrayList<BoltDef>();
+ retval.addAll(this.boltMap.values());
+ return retval;
+ }
+
+ public void setBolts(List<BoltDef> bolts) {
+ this.boltMap = new LinkedHashMap<String, BoltDef>();
+ for(BoltDef bolt : bolts){
+ this.boltMap.put(bolt.getId(), bolt);
+ }
+ }
+
+ public List<StreamDef> getStreams() {
+ return streams;
+ }
+
+ public void setStreams(List<StreamDef> streams) {
+ this.streams = streams;
+ }
+
+ public Map<String, Object> getConfig() {
+ return config;
+ }
+
+ public void setConfig(Map<String, Object> config) {
+ this.config = config;
+ }
+
+ public List<BeanDef> getComponents() {
+ ArrayList<BeanDef> retval = new ArrayList<BeanDef>();
+ retval.addAll(this.componentMap.values());
+ return retval;
+ }
+
+ public void setComponents(List<BeanDef> components) {
+ this.componentMap = new LinkedHashMap<String, BeanDef>();
+ for(BeanDef component : components){
+ this.componentMap.put(component.getId(), component);
+ }
+ }
+
+ public List<IncludeDef> getIncludes() {
+ return includes;
+ }
+
+ public void setIncludes(List<IncludeDef> includes) {
+ this.includes = includes;
+ }
+
+ // utility methods
+ public int parallelismForBolt(String boltId){
+ return this.boltMap.get(boltId).getParallelism();
+ }
+
+ public BoltDef getBoltDef(String id){
+ return this.boltMap.get(id);
+ }
+
+ public SpoutDef getSpoutDef(String id){
+ return this.spoutMap.get(id);
+ }
+
+ public BeanDef getComponent(String id){
+ return this.componentMap.get(id);
+ }
+
+ // used by includes implementation
+ public void addAllBolts(List<BoltDef> bolts, boolean override){
+ for(BoltDef bolt : bolts){
+ String id = bolt.getId();
+ if(this.boltMap.get(id) == null || override) {
+ this.boltMap.put(bolt.getId(), bolt);
+ } else {
+ LOG.warn("Ignoring attempt to create bolt '{}' with override == false.", id);
+ }
+ }
+ }
+
+ public void addAllSpouts(List<SpoutDef> spouts, boolean override){
+ for(SpoutDef spout : spouts){
+ String id = spout.getId();
+ if(this.spoutMap.get(id) == null || override) {
+ this.spoutMap.put(spout.getId(), spout);
+ } else {
+ LOG.warn("Ignoring attempt to create spout '{}' with override == false.", id);
+ }
+ }
+ }
+
+ public void addAllComponents(List<BeanDef> components, boolean override) {
+ for(BeanDef bean : components){
+ String id = bean.getId();
+ if(this.componentMap.get(id) == null || override) {
+ this.componentMap.put(bean.getId(), bean);
+ } else {
+ LOG.warn("Ignoring attempt to create component '{}' with override == false.", id);
+ }
+ }
+ }
+
+ public void addAllStreams(List<StreamDef> streams, boolean override) {
+ //TODO figure out how we want to deal with overrides. Users may want to add streams even when overriding other
+ // properties. For now we just add them blindly which could lead to a potentially invalid topology.
+ this.streams.addAll(streams);
+ }
+
+ public TopologySourceDef getTopologySource() {
+ return topologySource;
+ }
+
+ public void setTopologySource(TopologySourceDef topologySource) {
+ this.topologySource = topologySource;
+ }
+
+ public boolean isDslTopology(){
+ return this.topologySource == null;
+ }
+
+
+ public boolean validate(){
+ boolean hasSpouts = this.spoutMap != null && this.spoutMap.size() > 0;
+ boolean hasBolts = this.boltMap != null && this.boltMap.size() > 0;
+ boolean hasStreams = this.streams != null && this.streams.size() > 0;
+ boolean hasSpoutsBoltsStreams = hasStreams && hasBolts && hasSpouts;
+ // you cant define a topologySource and a DSL topology at the same time...
+ if (!isDslTopology() && ((hasSpouts || hasBolts || hasStreams))) {
+ return false;
+ }
+ if(isDslTopology() && (hasSpouts && hasBolts && hasStreams)) {
+ return true;
+ }
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
new file mode 100644
index 0000000..d6a2f57
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+public class TopologySourceDef extends ObjectDef {
+ public static final String DEFAULT_METHOD_NAME = "getTopology";
+
+ private String methodName;
+
+ public TopologySourceDef(){
+ this.methodName = DEFAULT_METHOD_NAME;
+ }
+
+ public String getMethodName() {
+ return methodName;
+ }
+
+ public void setMethodName(String methodName) {
+ this.methodName = methodName;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
new file mode 100644
index 0000000..e71bcc2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * Abstract parent class of component definitions
+ * (spouts/bolts)
+ */
+public abstract class VertexDef extends BeanDef {
+
+ // default parallelism to 1 so if it's ommitted, the topology will still function.
+ private int parallelism = 1;
+
+ public int getParallelism() {
+ return parallelism;
+ }
+
+ public void setParallelism(int parallelism) {
+ this.parallelism = parallelism;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
new file mode 100644
index 0000000..35904a2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.parser;
+
+import org.apache.storm.flux.model.BoltDef;
+import org.apache.storm.flux.model.IncludeDef;
+import org.apache.storm.flux.model.SpoutDef;
+import org.apache.storm.flux.model.TopologyDef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.TypeDescription;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.Constructor;
+
+import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Properties;
+
+public class FluxParser {
+ private static final Logger LOG = LoggerFactory.getLogger(FluxParser.class);
+
+ private FluxParser(){}
+
+ // TODO refactor input stream processing (see parseResource() method).
+ public static TopologyDef parseFile(String inputFile, boolean dumpYaml, boolean processIncludes,
+ String propertiesFile, boolean envSub) throws IOException {
+
+ FileInputStream in = new FileInputStream(inputFile);
+ TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
+ in.close();
+
+ return topology;
+ }
+
+ public static TopologyDef parseResource(String resource, boolean dumpYaml, boolean processIncludes,
+ String propertiesFile, boolean envSub) throws IOException {
+
+ InputStream in = FluxParser.class.getResourceAsStream(resource);
+ TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
+ in.close();
+
+ return topology;
+ }
+
+ public static TopologyDef parseInputStream(InputStream inputStream, boolean dumpYaml, boolean processIncludes,
+ String propertiesFile, boolean envSub) throws IOException {
+
+ Yaml yaml = yaml();
+
+ if (inputStream == null) {
+ LOG.error("Unable to load input stream");
+ System.exit(1);
+ }
+
+ TopologyDef topology = loadYaml(yaml, inputStream, propertiesFile, envSub);
+
+ if (dumpYaml) {
+ dumpYaml(topology, yaml);
+ }
+
+ if (processIncludes) {
+ return processIncludes(yaml, topology, propertiesFile, envSub);
+ } else {
+ return topology;
+ }
+ }
+
+ private static TopologyDef loadYaml(Yaml yaml, InputStream in, String propsFile, boolean envSubstitution) throws IOException {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ LOG.info("loading YAML from input stream...");
+ int b = -1;
+ while((b = in.read()) != -1){
+ bos.write(b);
+ }
+
+ // TODO substitution implementation is not exactly efficient or kind to memory...
+ String str = bos.toString();
+ // properties file substitution
+ if(propsFile != null){
+ LOG.info("Performing property substitution.");
+ InputStream propsIn = new FileInputStream(propsFile);
+ Properties props = new Properties();
+ props.load(propsIn);
+ for(Object key : props.keySet()){
+ str = str.replace("${" + key + "}", props.getProperty((String)key));
+ }
+ } else {
+ LOG.info("Not performing property substitution.");
+ }
+
+ // environment variable substitution
+ if(envSubstitution){
+ LOG.info("Performing environment variable substitution...");
+ Map<String, String> envs = System.getenv();
+ for(String key : envs.keySet()){
+ str = str.replace("${ENV-" + key + "}", envs.get(key));
+ }
+ } else {
+ LOG.info("Not performing environment variable substitution.");
+ }
+ return (TopologyDef)yaml.load(str);
+ }
+
+ private static void dumpYaml(TopologyDef topology, Yaml yaml){
+ System.out.println("Configuration (interpreted): \n" + yaml.dump(topology));
+ }
+
+ private static Yaml yaml(){
+ Constructor constructor = new Constructor(TopologyDef.class);
+
+ TypeDescription topologyDescription = new TypeDescription(TopologyDef.class);
+ topologyDescription.putListPropertyType("spouts", SpoutDef.class);
+ topologyDescription.putListPropertyType("bolts", BoltDef.class);
+ topologyDescription.putListPropertyType("includes", IncludeDef.class);
+ constructor.addTypeDescription(topologyDescription);
+
+ Yaml yaml = new Yaml(constructor);
+ return yaml;
+ }
+
+ /**
+ *
+ * @param yaml the yaml parser for parsing the include file(s)
+ * @param topologyDef the topology definition containing (possibly zero) includes
+ * @return The TopologyDef with includes resolved.
+ */
+ private static TopologyDef processIncludes(Yaml yaml, TopologyDef topologyDef, String propsFile, boolean envSub)
+ throws IOException {
+ //TODO support multiple levels of includes
+ if(topologyDef.getIncludes() != null) {
+ for (IncludeDef include : topologyDef.getIncludes()){
+ TopologyDef includeTopologyDef = null;
+ if (include.isResource()) {
+ LOG.info("Loading includes from resource: {}", include.getFile());
+ includeTopologyDef = parseResource(include.getFile(), true, false, propsFile, envSub);
+ } else {
+ LOG.info("Loading includes from file: {}", include.getFile());
+ includeTopologyDef = parseFile(include.getFile(), true, false, propsFile, envSub);
+ }
+
+ // if overrides are disabled, we won't replace anything that already exists
+ boolean override = include.isOverride();
+ // name
+ if(includeTopologyDef.getName() != null){
+ topologyDef.setName(includeTopologyDef.getName(), override);
+ }
+
+ // config
+ if(includeTopologyDef.getConfig() != null) {
+ //TODO move this logic to the model class
+ Map<String, Object> config = topologyDef.getConfig();
+ Map<String, Object> includeConfig = includeTopologyDef.getConfig();
+ if(override) {
+ config.putAll(includeTopologyDef.getConfig());
+ } else {
+ for(String key : includeConfig.keySet()){
+ if(config.containsKey(key)){
+ LOG.warn("Ignoring attempt to set topology config property '{}' with override == false", key);
+ }
+ else {
+ config.put(key, includeConfig.get(key));
+ }
+ }
+ }
+ }
+
+ //component overrides
+ if(includeTopologyDef.getComponents() != null){
+ topologyDef.addAllComponents(includeTopologyDef.getComponents(), override);
+ }
+ //bolt overrides
+ if(includeTopologyDef.getBolts() != null){
+ topologyDef.addAllBolts(includeTopologyDef.getBolts(), override);
+ }
+ //spout overrides
+ if(includeTopologyDef.getSpouts() != null) {
+ topologyDef.addAllSpouts(includeTopologyDef.getSpouts(), override);
+ }
+ //stream overrides
+ //TODO streams should be uniquely identifiable
+ if(includeTopologyDef.getStreams() != null) {
+ topologyDef.addAllStreams(includeTopologyDef.getStreams(), override);
+ }
+ } // end include processing
+ }
+ return topologyDef;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/resources/splash.txt
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/resources/splash.txt b/flux/flux-core/src/main/resources/splash.txt
new file mode 100644
index 0000000..337931a
--- /dev/null
+++ b/flux/flux-core/src/main/resources/splash.txt
@@ -0,0 +1,9 @@
+\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557 \u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557
+\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
+\u2588\u2588\u2588\u2588\u2588\u2557 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
+\u2588\u2588\u2554\u2550\u2550\u255d \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
+\u2588\u2588\u2551 \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
+\u255a\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d \u255a\u2550\u255d
++- Apache Storm -+
++- data FLow User eXperience -+
+Version: ${project.version}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
new file mode 100644
index 0000000..ff67867
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class FluxBuilderTest {
+
+ @Test
+ public void testIsPrimitiveNumber() throws Exception {
+ assertTrue(FluxBuilder.isPrimitiveNumber(int.class));
+ assertFalse(FluxBuilder.isPrimitiveNumber(boolean.class));
+ assertFalse(FluxBuilder.isPrimitiveNumber(String.class));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
new file mode 100644
index 0000000..c5807f8
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux;
+
+import org.junit.Test;
+
+public class IntegrationTest {
+
+ private static boolean skipTest = true;
+
+ static {
+ String skipStr = System.getProperty("skipIntegration");
+ if(skipStr != null && skipStr.equalsIgnoreCase("false")){
+ skipTest = false;
+ }
+ }
+
+ @Test
+ public void testRunTopologySource() throws Exception {
+ if(!skipTest) {
+ Flux.main(new String[]{"-s", "30000", "src/test/resources/configs/existing-topology.yaml"});
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
new file mode 100644
index 0000000..c9227f6
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.flux.model.ExecutionContext;
+import org.apache.storm.flux.model.TopologyDef;
+import org.apache.storm.flux.parser.FluxParser;
+import org.apache.storm.flux.test.TestBolt;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TCKTest {
+ @Test
+ public void testTCK() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/tck.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testShellComponents() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/shell_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testBadShellComponents() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_shell_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testKafkaSpoutConfig() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testLoadFromResource() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+
+ @Test
+ public void testHdfs() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/hdfs_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testDiamondTopology() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/diamond-topology.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+
+ @Test
+ public void testHbase() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/simple_hbase.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testBadHbase() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_hbase.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testIncludes() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/include_test.yaml", false, true, null, false);
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ assertTrue(topologyDef.getName().equals("include-topology"));
+ assertTrue(topologyDef.getBolts().size() > 0);
+ assertTrue(topologyDef.getSpouts().size() > 0);
+ topology.validate();
+ }
+
+ @Test
+ public void testTopologySource() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testTopologySourceWithReflection() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testTopologySourceWithConfigParam() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection-config.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testTopologySourceWithMethodName() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-method-override.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+
+ @Test
+ public void testTridentTopologySource() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-trident.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testInvalidTopologySource() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/invalid-existing-topology.yaml", false, true, null, false);
+ assertFalse("Topology config is invalid.", topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ }
+
+
+ @Test
+ public void testTopologySourceWithGetMethodName() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+ }
+
+ @Test
+ public void testTopologySourceWithConfigMethods() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/config-methods-test.yaml", false, true, null, false);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+
+ // make sure the property was actually set
+ TestBolt bolt = (TestBolt)context.getBolt("bolt-1");
+ assertTrue(bolt.getFoo().equals("foo"));
+ assertTrue(bolt.getBar().equals("bar"));
+ assertTrue(bolt.getFooBar().equals("foobar"));
+ assertArrayEquals(new TestBolt.TestClass[] {new TestBolt.TestClass("foo"), new TestBolt.TestClass("bar"), new TestBolt.TestClass("baz")}, bolt.getClasses());
+ }
+
+ @Test
+ public void testVariableSubstitution() throws Exception {
+ TopologyDef topologyDef = FluxParser.parseResource("/configs/substitution-test.yaml", false, true, "src/test/resources/configs/test.properties", true);
+ assertTrue(topologyDef.validate());
+ Config conf = FluxBuilder.buildConfig(topologyDef);
+ ExecutionContext context = new ExecutionContext(topologyDef, conf);
+ StormTopology topology = FluxBuilder.buildTopology(context);
+ assertNotNull(topology);
+ topology.validate();
+
+ // test basic substitution
+ assertEquals("Property not replaced.",
+ "substitution-topology",
+ context.getTopologyDef().getName());
+
+ // test environment variable substitution
+ // $PATH should be defined on most systems
+ String envPath = System.getenv().get("PATH");
+ assertEquals("ENV variable not replaced.",
+ envPath,
+ context.getTopologyDef().getConfig().get("test.env.value"));
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
new file mode 100644
index 0000000..dcded17
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.multilang;
+
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Sanity checks to make sure we can at least invoke the shells used.
+ */
+public class MultilangEnvirontmentTest {
+ private static final Logger LOG = LoggerFactory.getLogger(MultilangEnvirontmentTest.class);
+
+ @Test
+ public void testInvokePython() throws Exception {
+ String[] command = new String[]{"python", "--version"};
+ int exitVal = invokeCommand(command);
+ assertEquals("Exit value for python is 0.", 0, exitVal);
+ }
+
+ @Test
+ public void testInvokeNode() throws Exception {
+ String[] command = new String[]{"node", "--version"};
+ int exitVal = invokeCommand(command);
+ assertEquals("Exit value for node is 0.", 0, exitVal);
+ }
+
+ private static class StreamRedirect implements Runnable {
+ private InputStream in;
+ private OutputStream out;
+
+ public StreamRedirect(InputStream in, OutputStream out) {
+ this.in = in;
+ this.out = out;
+ }
+
+ @Override
+ public void run() {
+ try {
+ int i = -1;
+ while ((i = this.in.read()) != -1) {
+ out.write(i);
+ }
+ this.in.close();
+ this.out.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ private int invokeCommand(String[] args) throws Exception {
+ LOG.debug("Invoking command: {}", args);
+
+ ProcessBuilder pb = new ProcessBuilder(args);
+ pb.redirectErrorStream(true);
+ final Process proc = pb.start();
+
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ Thread t = new Thread(new StreamRedirect(proc.getInputStream(), out));
+ t.start();
+ int exitVal = proc.waitFor();
+ LOG.debug("Command result: {}", out.toString());
+ return exitVal;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
new file mode 100644
index 0000000..ff65a8a
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+import java.util.Map;
+
+/**
+ * Test topology source that does not implement TopologySource, but has the same
+ * `getTopology()` method.
+ */
+public class SimpleTopology{
+
+
+ public SimpleTopology(){}
+
+ public SimpleTopology(String foo, String bar){}
+
+ public StormTopology getTopologyWithDifferentMethodName(Map<String, Object> config){
+ return getTopology(config);
+ }
+
+
+ public StormTopology getTopology(Map<String, Object> config) {
+ TopologyBuilder builder = new TopologyBuilder();
+
+ // spouts
+ FluxShellSpout spout = new FluxShellSpout(
+ new String[]{"node", "randomsentence.js"},
+ new String[]{"word"});
+ builder.setSpout("sentence-spout", spout, 1);
+
+ // bolts
+ builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+ .shuffleGrouping("sentence-spout");
+
+ return builder.createTopology();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
new file mode 100644
index 0000000..2fadacf
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.api.TopologySource;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+import java.util.Map;
+
+public class SimpleTopologySource implements TopologySource {
+
+
+ public SimpleTopologySource(){}
+
+ public SimpleTopologySource(String foo, String bar){}
+
+
+ @Override
+ public StormTopology getTopology(Map<String, Object> config) {
+ TopologyBuilder builder = new TopologyBuilder();
+
+ // spouts
+ FluxShellSpout spout = new FluxShellSpout(
+ new String[]{"node", "randomsentence.js"},
+ new String[]{"word"});
+ builder.setSpout("sentence-spout", spout, 1);
+
+ // bolts
+ builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+ .shuffleGrouping("sentence-spout");
+
+ return builder.createTopology();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
new file mode 100644
index 0000000..78195b5
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+/**
+ * Test topology source that does not implement TopologySource, but has the same
+ * `getTopology()` method.
+ */
+public class SimpleTopologyWithConfigParam {
+
+
+ public SimpleTopologyWithConfigParam(){}
+
+ public SimpleTopologyWithConfigParam(String foo, String bar){}
+
+
+ public StormTopology getTopology(Config config) {
+ TopologyBuilder builder = new TopologyBuilder();
+
+ // spouts
+ FluxShellSpout spout = new FluxShellSpout(
+ new String[]{"node", "randomsentence.js"},
+ new String[]{"word"});
+ builder.setSpout("sentence-spout", spout, 1);
+
+ // bolts
+ builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+ .shuffleGrouping("sentence-spout");
+
+ return builder.createTopology();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
new file mode 100644
index 0000000..28d11b6
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+
+public class TestBolt extends BaseBasicBolt {
+ private static final Logger LOG = LoggerFactory.getLogger(TestBolt.class);
+
+ private String foo;
+ private String bar;
+ private String fooBar;
+ private String none;
+ private TestClass[] classes;
+
+ public static class TestClass implements Serializable {
+ private String field;
+
+ public TestClass(String field) {
+ this.field = field;
+ }
+
+ public String getField() {
+ return field;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (!(o instanceof TestClass)) return false;
+
+ TestClass testClass = (TestClass) o;
+
+ return getField() != null ? getField().equals(testClass.getField()) : testClass.getField() == null;
+ }
+
+ @Override
+ public int hashCode() {
+ return getField() != null ? getField().hashCode() : 0;
+ }
+ }
+
+
+ public static enum TestEnum {
+ FOO,
+ BAR
+ }
+
+ public TestBolt(TestEnum te){
+
+ }
+
+ public TestBolt(TestEnum te, float f){
+
+ }
+
+ public TestBolt(TestEnum te, float f, boolean b){
+
+ }
+
+ public TestBolt(TestEnum te, float f, boolean b, TestClass... str) {
+
+ }
+
+ @Override
+ public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
+ LOG.info("{}", tuple);
+ }
+
+ @Override
+ public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+
+ }
+
+ // config methods
+ public void withFoo(String foo){
+ this.foo = foo;
+ }
+ public void withNone(){
+ this.none = "hit";
+ }
+ public void withBar(String bar){
+ this.bar = bar;
+ }
+
+ public void withFooBar(String foo, String bar){
+ this.fooBar = foo + bar;
+ }
+
+ public void withClasses(TestClass...classes) {
+ this.classes = classes;
+ }
+
+ public String getFoo(){
+ return this.foo;
+ }
+ public String getBar(){
+ return this.bar;
+ }
+
+ public String getFooBar(){
+ return this.fooBar;
+ }
+
+ public TestClass[] getClasses() {
+ return classes;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
new file mode 100644
index 0000000..36b272b
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.kafka.StringScheme;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.operation.BaseFunction;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.operation.builtin.Count;
+import org.apache.storm.trident.testing.FixedBatchSpout;
+import org.apache.storm.trident.testing.MemoryMapState;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+/**
+ * Basic Trident example that will return a `StormTopology` from a `getTopology()` method.
+ */
+public class TridentTopologySource {
+
+ private FixedBatchSpout spout;
+
+ public StormTopology getTopology(Config config) {
+
+ this.spout = new FixedBatchSpout(new Fields("sentence"), 20,
+ new Values("one two"),
+ new Values("two three"),
+ new Values("three four"),
+ new Values("four five"),
+ new Values("five six")
+ );
+
+
+ TridentTopology trident = new TridentTopology();
+
+ trident.newStream("wordcount", spout).name("sentence").parallelismHint(1).shuffle()
+ .each(new Fields("sentence"), new Split(), new Fields("word"))
+ .parallelismHint(1)
+ .groupBy(new Fields("word"))
+ .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count"))
+ .parallelismHint(1);
+ return trident.build();
+ }
+
+ public static class Split extends BaseFunction {
+ @Override
+ public void execute(TridentTuple tuple, TridentCollector collector) {
+ String sentence = tuple.getString(0);
+ for (String word : sentence.split(" ")) {
+ collector.emit(new Values(word));
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/bad_hbase.yaml b/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
new file mode 100644
index 0000000..a29e314
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
@@ -0,0 +1,98 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hbase-wordcount"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+
+components:
+ - id: "columnFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ - ["word"]
+
+ - id: "counterFields"
+ className: "org.apache.storm.tuple.Fields"
+ constructorArgs:
+ # !!! the following won't work, and should thow an IllegalArgumentException...
+ - "count"
+
+ - id: "mapper"
+ className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+ configMethods:
+ - name: "withRowKeyField"
+ args: ["word"]
+ - name: "withColumnFields"
+ args: [ref: "columnFields"]
+ - name: "withCounterFields"
+ args: [ref: "counterFields"]
+ - name: "withColumnFamily"
+ args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ hbase.conf:
+ hbase.rootdir: "hdfs://hadoop:54310/hbase"
+ hbase.zookeeper.quorum: "hadoop"
+
+# spout definitions
+spouts:
+ - id: "word-spout"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+
+bolts:
+ - id: "count-bolt"
+ className: "org.apache.storm.testing.TestWordCounter"
+
+ - id: "hbase-bolt"
+ className: "org.apache.storm.hbase.bolt.HBaseBolt"
+ constructorArgs:
+ - "WordCount" # HBase table name
+ - ref: "mapper"
+ configMethods:
+ - name: "withConfigKey"
+ args: ["hbase.conf"]
+ parallelism: 1
+
+
+streams:
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "word-spout"
+ to: "count-bolt"
+ grouping:
+ type: SHUFFLE
+
+ - name: "" # name isn't used (placeholder for logging, UI, etc.)
+ from: "count-bolt"
+ to: "hbase-bolt"
+ grouping:
+ type: FIELDS
+ args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml b/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
new file mode 100644
index 0000000..0892ce7
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
@@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+# - id: "myComponent"
+# className: "com.foo.bar.MyComponent"
+# constructorArgs:
+# - ...
+# properties:
+# foo: "bar"
+# bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+ # ...
+
+# spout definitions
+spouts:
+ - id: "sentence-spout"
+ className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+ # shell spout constructor takes 2 arguments: String[], String[]
+ constructorArgs:
+ # command line
+ - ["node", "randomsentence.js"]
+ # output fields
+ - ["word"]
+ configMethods:
+ - name: "addComponentConfig"
+ args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
+ - name: "addComponentConfig"
+ args:
+ - "publisher.data_paths"
+ - ["actions", "hello"]
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "splitsentence"
+ className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+ constructorArgs:
+ # command line
+ - ["python", "splitsentence.py"]
+ # output fields
+ - ["word"]
+ configMethods:
+ - name: "addComponentConfig"
+ args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
+ - name: "addComponentConfig"
+ args:
+ - "publisher.data_paths"
+ - ["actions", "hello"]
+ parallelism: 1
+ # ...
+
+ - id: "log"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+ # ...
+
+ - id: "count"
+ className: "org.apache.storm.testing.TestWordCounter"
+ parallelism: 1
+ # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+ - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+ from: "sentence-spout"
+ to: "splitsentence"
+ grouping:
+ type: SHUFFLE
+
+ - name: "split --> count"
+ from: "splitsentence"
+ to: "count"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - name: "count --> log"
+ from: "count"
+ to: "log"
+ grouping:
+ type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/config-methods-test.yaml b/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
new file mode 100644
index 0000000..7c4ffb3
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
@@ -0,0 +1,92 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+name: "yaml-topology"
+
+#
+config:
+ topology.workers: 1
+ # ...
+
+components:
+ - id: "foo"
+ className: "org.apache.storm.flux.test.TestBolt$TestClass"
+ constructorArgs:
+ - "foo"
+ - id: "bar"
+ className: "org.apache.storm.flux.test.TestBolt$TestClass"
+ constructorArgs:
+ - "bar"
+ - id: "baz"
+ className: "org.apache.storm.flux.test.TestBolt$TestClass"
+ constructorArgs:
+ - "baz"
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+ # ...
+
+# bolt definitions
+bolts:
+ - id: "bolt-1"
+ className: "org.apache.storm.flux.test.TestBolt"
+ parallelism: 1
+ constructorArgs:
+ - FOO # enum class
+ - 1.0
+ - true
+ - reflist: ["foo", "bar"]
+ configMethods:
+ - name: "withFoo"
+ args:
+ - "foo"
+ - name: "withNone"
+ - name: "withBar"
+ args:
+ - "bar"
+ - name: "withFooBar"
+ args:
+ - "foo"
+ - "bar"
+ - name: "withClasses"
+ args:
+ - reflist:
+ - "foo"
+ - "bar"
+ - "baz"
+
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "spout-1"
+ to: "bolt-1"
+ grouping:
+ type: SHUFFLE
+
+
+
+
+
+
+
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/diamond-topology.yaml b/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
new file mode 100644
index 0000000..957c258
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
@@ -0,0 +1,87 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "diamond-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+ topology.workers: 1
+
+# spout definitions
+spouts:
+ - id: "spout-1"
+ className: "org.apache.storm.testing.TestWordSpout"
+ parallelism: 1
+
+# bolt definitions
+bolts:
+ - id: "A"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+ - id: "B"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+ - id: "C"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+ - id: "D"
+ className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+ parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+ - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+# id: "connection-1"
+ from: "spout-1"
+ to: "A"
+ grouping:
+ type: FIELDS
+ args: ["word"]
+
+ - from: "A"
+ to: "B"
+ grouping:
+ type: SHUFFLE
+
+ - from: "A"
+ to: "C"
+ grouping:
+ type: SHUFFLE
+
+ - from: "C"
+ to: "D"
+ grouping:
+ type: SHUFFLE
+
+ - from: "B"
+ to: "D"
+ grouping:
+ type: SHUFFLE
+
+
+
+
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
new file mode 100644
index 0000000..fceeeed
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopology"
+ methodName: "getTopologyWithDifferentMethodName"
+ constructorArgs:
+ - "foo"
+ - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
new file mode 100644
index 0000000..440fe4d
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
@@ -0,0 +1,24 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+ className: "org.apache.storm.flux.test.SimpleTopologyWithConfigParam"
+ constructorArgs:
+ - "foo"
+ - "bar"
\ No newline at end of file
[15/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
deleted file mode 100644
index e146069..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- *
- * * Licensed to the Apache Software Foundation (ASF) under one
- * * or more contributor license agreements. See the NOTICE file
- * * distributed with this work for additional information
- * * regarding copyright ownership. The ASF licenses this file
- * * to you under the Apache License, Version 2.0 (the
- * * "License"); you may not use this file except in compliance
- * * with the License. You may obtain a copy of the License at
- * *
- * * http://www.apache.org/licenses/LICENSE-2.0
- * *
- * * Unless required by applicable law or agreed to in writing, software
- * * distributed under the License is distributed on an "AS IS" BASIS,
- * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * * See the License for the specific language governing permissions and
- * * limitations under the License.
- *
- */
-package org.apache.storm.sql.planner.trident;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
-import org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
-import org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.calcite.rel.rules.PruneEmptyRules;
-import org.apache.calcite.rel.rules.ReduceExpressionsRule;
-import org.apache.calcite.rel.rules.SortRemoveRule;
-import org.apache.calcite.rel.rules.UnionEliminatorRule;
-import org.apache.calcite.rel.stream.StreamRules;
-import org.apache.calcite.tools.RuleSet;
-import org.apache.storm.sql.planner.trident.rules.TridentCalcRule;
-import org.apache.storm.sql.planner.trident.rules.TridentFilterRule;
-import org.apache.storm.sql.planner.trident.rules.TridentScanRule;
-import org.apache.storm.sql.planner.trident.rules.TridentAggregateRule;
-import org.apache.storm.sql.planner.trident.rules.TridentJoinRule;
-import org.apache.storm.sql.planner.trident.rules.TridentModifyRule;
-import org.apache.storm.sql.planner.trident.rules.TridentProjectRule;
-
-import java.util.Iterator;
-
-public class TridentStormRuleSets {
- private static final ImmutableSet<RelOptRule> calciteToStormConversionRules =
- ImmutableSet.<RelOptRule>builder().add(
- SortRemoveRule.INSTANCE,
-
- FilterToCalcRule.INSTANCE,
- ProjectToCalcRule.INSTANCE,
- FilterCalcMergeRule.INSTANCE,
- ProjectCalcMergeRule.INSTANCE,
- CalcMergeRule.INSTANCE,
-
- PruneEmptyRules.FILTER_INSTANCE,
- PruneEmptyRules.PROJECT_INSTANCE,
- PruneEmptyRules.UNION_INSTANCE,
-
- ProjectFilterTransposeRule.INSTANCE,
- FilterProjectTransposeRule.INSTANCE,
- ProjectRemoveRule.INSTANCE,
-
- ReduceExpressionsRule.FILTER_INSTANCE,
- ReduceExpressionsRule.PROJECT_INSTANCE,
- ReduceExpressionsRule.CALC_INSTANCE,
-
- // merge and push unions rules
- UnionEliminatorRule.INSTANCE,
-
- TridentScanRule.INSTANCE,
- TridentFilterRule.INSTANCE,
- TridentProjectRule.INSTANCE,
- TridentAggregateRule.INSTANCE,
- TridentJoinRule.INSTANCE,
- TridentModifyRule.INSTANCE,
- TridentCalcRule.INSTANCE
- ).build();
-
- public static RuleSet[] getRuleSets() {
- return new RuleSet[]{
- new StormRuleSet(StreamRules.RULES),
- new StormRuleSet(ImmutableSet.<RelOptRule>builder().addAll(StreamRules.RULES).addAll(calciteToStormConversionRules).build())
- };
- }
-
- private static class StormRuleSet implements RuleSet {
- final ImmutableSet<RelOptRule> rules;
-
- public StormRuleSet(ImmutableSet<RelOptRule> rules) {
- this.rules = rules;
- }
-
- public StormRuleSet(ImmutableList<RelOptRule> rules) {
- this.rules = ImmutableSet.<RelOptRule>builder()
- .addAll(rules)
- .build();
- }
-
- @Override
- public Iterator<RelOptRule> iterator() {
- return rules.iterator();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
deleted file mode 100644
index 482e841..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import com.google.common.collect.Lists;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormCalcRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationCalc;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class TridentCalcRel extends StormCalcRelBase implements TridentRel {
- public TridentCalcRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
- super(cluster, traits, child, program);
- }
-
- @Override
- public Calc copy(RelTraitSet traitSet, RelNode child, RexProgram program) {
- return new TridentCalcRel(getCluster(), traitSet, child, program);
- }
-
- @Override
- public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
- // SingleRel
- RelNode input = getInput();
- StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
- Stream inputStream = planCreator.pop().toStream();
-
- String stageName = StormRelUtils.getStageName(this);
-
- RelDataType inputRowType = getInput(0).getRowType();
-
- List<String> outputFieldNames = getRowType().getFieldNames();
- int outputCount = outputFieldNames.size();
-
- // filter
- ExecutableExpression filterInstance = null;
- RexLocalRef condition = program.getCondition();
- if (condition != null) {
- RexNode conditionNode = program.expandLocalRef(condition);
- filterInstance = planCreator.createScalarInstance(Lists.newArrayList(conditionNode), inputRowType,
- StormRelUtils.getClassName(this));
- }
-
- // projection
- ExecutableExpression projectionInstance = null;
- List<RexLocalRef> projectList = program.getProjectList();
- if (projectList != null && !projectList.isEmpty()) {
- List<RexNode> expandedNodes = new ArrayList<>();
- for (RexLocalRef project : projectList) {
- expandedNodes.add(program.expandLocalRef(project));
- }
-
- projectionInstance = planCreator.createScalarInstance(expandedNodes, inputRowType,
- StormRelUtils.getClassName(this));
- }
-
- if (projectionInstance == null && filterInstance == null) {
- // it shouldn't be happen
- throw new IllegalStateException("Either projection or condition, or both should be provided.");
- }
-
- final Stream finalStream = inputStream
- .flatMap(new EvaluationCalc(filterInstance, projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
- .name(stageName);
-
- planCreator.addStream(finalStream);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
deleted file mode 100644
index 1fe0927..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormFilterRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationFilter;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.List;
-
-public class TridentFilterRel extends StormFilterRelBase implements TridentRel {
- public TridentFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
- super(cluster, traits, child, condition);
- }
-
- @Override
- public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
- return new TridentFilterRel(getCluster(), traitSet, input, condition);
- }
-
- @Override
- public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
- // SingleRel
- RelNode input = getInput();
- StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
- Stream inputStream = planCreator.pop().toStream();
-
- String stageName = StormRelUtils.getStageName(this);
-
- List<RexNode> childExps = getChildExps();
- RelDataType inputRowType = getInput(0).getRowType();
-
- String filterClassName = StormRelUtils.getClassName(this);
- ExecutableExpression filterInstance = planCreator.createScalarInstance(childExps, inputRowType, filterClassName);
-
- IAggregatableStream finalStream = inputStream.filter(new EvaluationFilter(filterInstance, planCreator.getDataContext()))
- .name(stageName);
- planCreator.addStream(finalStream);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
deleted file mode 100644
index d221498..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-
-public enum TridentLogicalConvention implements Convention {
- INSTANCE;
-
- @Override
- public Class getInterface() {
- return TridentRel.class;
- }
-
- @Override
- public String getName() {
- return "STORM_LOGICAL";
- }
-
- @Override
- public RelTraitDef getTraitDef() {
- return ConventionTraitDef.INSTANCE;
- }
-
- @Override
- public boolean satisfies(RelTrait trait) {
- return this == trait;
- }
-
- @Override
- public void register(RelOptPlanner planner) {}
-
- @Override
- public String toString() {
- return getName();
- }
-
- @Override
- public boolean canConvertConvention(Convention toConvention) {
- return false;
- }
-
- @Override
- public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
deleted file mode 100644
index 06be5d7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormProjectRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationFunction;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-
-public class TridentProjectRel extends StormProjectRelBase implements TridentRel {
- public TridentProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
- super(cluster, traits, input, projects, rowType);
- }
-
- @Override
- public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
- return new TridentProjectRel(getCluster(), traitSet, input, projects, rowType);
- }
-
- @Override
- public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
- // SingleRel
- RelNode input = getInput();
- StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
- Stream inputStream = planCreator.pop().toStream();
-
- String stageName = StormRelUtils.getStageName(this);
- String projectionClassName = StormRelUtils.getClassName(this);
-
- List<String> outputFieldNames = getRowType().getFieldNames();
- int outputCount = outputFieldNames.size();
-
- List<RexNode> childExps = getChildExps();
- RelDataType inputRowType = getInput(0).getRowType();
-
- ExecutableExpression projectionInstance = planCreator.createScalarInstance(childExps, inputRowType, projectionClassName);
- Stream finalStream = inputStream
- .map(new EvaluationFunction(projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
- .name(stageName);
-
- planCreator.addStream(finalStream);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
deleted file mode 100644
index fa92ec9..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-
-public interface TridentRel extends StormRelNode {
- void tridentPlan(TridentPlanCreator planCreator) throws Exception;
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
deleted file mode 100644
index e92c29b..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormStreamInsertRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-
-public class TridentStreamInsertRel extends StormStreamInsertRelBase implements TridentRel {
- public TridentStreamInsertRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
- super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
- }
-
- @Override
- public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
- return new TridentStreamInsertRel(getCluster(), traitSet, getTable(), getCatalogReader(),
- sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
- }
-
- @Override
- public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
- // SingleRel
- RelNode input = getInput();
- StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
- Stream inputStream = planCreator.pop().toStream();
-
- String stageName = StormRelUtils.getStageName(this);
-
- Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported.");
-
- List<String> inputFields = this.input.getRowType().getFieldNames();
- List<String> outputFields = getRowType().getFieldNames();
-
- // FIXME: this should be really different...
- String tableName = Joiner.on('.').join(getTable().getQualifiedName());
- ISqlTridentDataSource.SqlTridentConsumer consumer = planCreator.getSources().get(tableName).getConsumer();
-
- // In fact this is normally the end of stream, but I'm still not sure so I open new streams based on State values
- IAggregatableStream finalStream = inputStream
- .partitionPersist(consumer.getStateFactory(), new Fields(inputFields), consumer.getStateUpdater(),
- new Fields(outputFields))
- .newValuesStream().name(stageName);
-
- planCreator.addStream(finalStream);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
deleted file mode 100644
index c563d73..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rel;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormStreamScanRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.Map;
-
-public class TridentStreamScanRel extends StormStreamScanRelBase implements TridentRel {
- private final int parallelismHint;
-
- public TridentStreamScanRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table, int parallelismHint) {
- super(cluster, traitSet, table);
- this.parallelismHint = parallelismHint;
- }
-
- @Override
- public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
- String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
- // FIXME: this should be really different...
- Map<String, ISqlTridentDataSource> sources = planCreator.getSources();
- if (!sources.containsKey(sourceName)) {
- throw new RuntimeException("Cannot find table " + sourceName);
- }
-
- String stageName = StormRelUtils.getStageName(this);
- IAggregatableStream finalStream = planCreator.getTopology().newStream(stageName, sources.get(sourceName).getProducer())
- .parallelismHint(parallelismHint);
- planCreator.addStream(finalStream);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
deleted file mode 100644
index ac35414..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentAggregateRule extends ConverterRule {
- public static final RelOptRule INSTANCE = new TridentAggregateRule();
-
- private TridentAggregateRule() {
- super(LogicalAggregate.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentAggregateRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- throw new UnsupportedOperationException("Aggregate operation is not supported.");
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
deleted file mode 100644
index 25126ec..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.storm.sql.planner.trident.rel.TridentCalcRel;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentCalcRule extends ConverterRule {
- public static final TridentCalcRule INSTANCE = new TridentCalcRule();
-
- private TridentCalcRule() {
- super(LogicalCalc.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentCalcRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- final Calc calc = (Calc) rel;
- final RelNode input = calc.getInput();
-
- return new TridentCalcRel(calc.getCluster(), calc.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
- convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
- calc.getProgram());
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
deleted file mode 100644
index 7f9c41f..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.storm.sql.planner.trident.rel.TridentFilterRel;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentFilterRule extends ConverterRule {
- public static TridentFilterRule INSTANCE = new TridentFilterRule();
-
- private TridentFilterRule() {
- super(LogicalFilter.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentFilterRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- final Filter filter = (Filter) rel;
- final RelNode input = filter.getInput();
-
- return new TridentFilterRel(filter.getCluster(),
- filter.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
- convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
- filter.getCondition());
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
deleted file mode 100644
index 90f5083..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentJoinRule extends ConverterRule {
- public static final TridentJoinRule INSTANCE = new TridentJoinRule();
-
- private TridentJoinRule() {
- super(LogicalJoin.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentJoinRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- throw new UnsupportedOperationException("Join operation is not supported.");
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
deleted file mode 100644
index 2155451..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.Table;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentStreamInsertRel;
-
-import java.util.List;
-
-public class TridentModifyRule extends ConverterRule {
- public static final TridentModifyRule INSTANCE = new TridentModifyRule();
-
- private TridentModifyRule() {
- super(LogicalTableModify.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentModifyRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- final TableModify tableModify = (TableModify) rel;
- final RelNode input = tableModify.getInput();
-
- final RelOptCluster cluster = tableModify.getCluster();
- final RelTraitSet traitSet = tableModify.getTraitSet().replace(TridentLogicalConvention.INSTANCE);
- final RelOptTable relOptTable = tableModify.getTable();
- final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
- final RelNode convertedInput = convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE));
- final TableModify.Operation operation = tableModify.getOperation();
- final List<String> updateColumnList = tableModify.getUpdateColumnList();
- final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
- final boolean flattened = tableModify.isFlattened();
-
- final Table table = tableModify.getTable().unwrap(Table.class);
-
- switch (table.getJdbcTableType()) {
- case STREAM:
- if (operation != TableModify.Operation.INSERT) {
- throw new UnsupportedOperationException(String.format("Streams doesn't support %s modify operation", operation));
- }
- return new TridentStreamInsertRel(cluster, traitSet, relOptTable, catalogReader, convertedInput, operation,
- updateColumnList, sourceExpressionList, flattened);
- default:
- throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
deleted file mode 100644
index 2922725..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentProjectRel;
-
-public class TridentProjectRule extends ConverterRule {
- public static final TridentProjectRule INSTANCE = new TridentProjectRule();
-
- private TridentProjectRule() {
- super(LogicalProject.class, Convention.NONE, TridentLogicalConvention.INSTANCE,
- "TridentProjectRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- final Project project = (Project) rel;
- final RelNode input = project.getInput();
-
- return new TridentProjectRel(project.getCluster(),
- project.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
- convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)), project.getProjects(), project.getRowType());
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
deleted file mode 100644
index abbd680..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.planner.trident.rules;
-
-import org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.schema.Table;
-import org.apache.storm.sql.calcite.ParallelStreamableTable;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentStreamScanRel;
-
-public class TridentScanRule extends ConverterRule {
- public static final TridentScanRule INSTANCE = new TridentScanRule();
- public static final int DEFAULT_PARALLELISM_HINT = 1;
-
- private TridentScanRule() {
- super(EnumerableTableScan.class, EnumerableConvention.INSTANCE, TridentLogicalConvention.INSTANCE, "TridentScanRule");
- }
-
- @Override
- public RelNode convert(RelNode rel) {
- final TableScan scan = (TableScan) rel;
- int parallelismHint = DEFAULT_PARALLELISM_HINT;
-
- final ParallelStreamableTable parallelTable = scan.getTable().unwrap(ParallelStreamableTable.class);
- if (parallelTable != null && parallelTable.parallelismHint() != null) {
- parallelismHint = parallelTable.parallelismHint();
- }
-
- final Table table = scan.getTable().unwrap(Table.class);
- switch (table.getJdbcTableType()) {
- case STREAM:
- return new TridentStreamScanRel(scan.getCluster(),
- scan.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
- scan.getTable(), parallelismHint);
- default:
- throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
deleted file mode 100644
index 82dc184..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
+++ /dev/null
@@ -1,492 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.tuple.Values;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public class TestStormSql {
- private static class MockDataSourceProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "mock";
- }
-
- @Override
- public DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- return new TestUtils.MockDataSource();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- return new TestUtils.MockSqlTridentDataSource();
- }
- }
-
- private static class MockNestedDataSourceProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "mocknested";
- }
-
- @Override
- public DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- return new TestUtils.MockNestedDataSource();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- return new TestUtils.MockSqlTridentDataSource();
- }
- }
-
- private static class MockGroupDataSourceProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "mockgroup";
- }
-
- @Override
- public DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- return new TestUtils.MockGroupDataSource();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- return new TestUtils.MockSqlTridentGroupedDataSource();
- }
- }
-
- private static class MockEmpDataSourceProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "mockemp";
- }
-
- @Override
- public DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- return new TestUtils.MockEmpDataSource();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- return new TestUtils.MockSqlTridentJoinDataSourceEmp();
- }
- }
-
- private static class MockDeptDataSourceProvider implements DataSourcesProvider {
- @Override
- public String scheme() {
- return "mockdept";
- }
-
- @Override
- public DataSource construct(
- URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- return new TestUtils.MockDeptDataSource();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- return new TestUtils.MockSqlTridentJoinDataSourceDept();
- }
- }
-
-
- @BeforeClass
- public static void setUp() {
- 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());
- }
-
- @AfterClass
- public static void tearDown() {
- DataSourcesRegistry.providerMap().remove("mock");
- DataSourcesRegistry.providerMap().remove("mocknested");
- }
-
- @Test
- public void testExternalDataSource() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
- stmt.add("SELECT STREAM ID + 1 FROM FOO WHERE ID > 2");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(2, values.size());
- Assert.assertEquals(4, values.get(0).get(0));
- Assert.assertEquals(5, values.get(1).get(0));
- }
-
- @Test
- public void testExternalDataSourceNested() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
- stmt.add("SELECT STREAM ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE CAST(MAPFIELD['b'] AS INTEGER) = 2 AND CAST(ARRAYFIELD[2] AS INTEGER) = 200");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- System.out.println(values);
- Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
- Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
- Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
- }
-
- @Test
- public void testExternalNestedNonExistKeyAccess() throws Exception {
- List<String> stmt = new ArrayList<>();
- // this triggers java.lang.RuntimeException: Cannot convert null to int
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
- stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE CAST(MAPFIELD['a'] AS INTEGER) = 2");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(0, values.size());
- }
-
- @Test
- public void testExternalNestedNonExistKeyAccess2() throws Exception {
- List<String> stmt = new ArrayList<>();
- // this triggers java.lang.RuntimeException: Cannot convert null to int
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
- stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE CAST(NESTEDMAPFIELD['b']['c'] AS INTEGER) = 4");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(0, values.size());
- }
-
- @Test
- public void testExternalNestedInvalidAccessStringIndexOnArray() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
- stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE CAST(ARRAYFIELD['a'] AS INTEGER) = 200");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(0, values.size());
- }
-
- @Test
- public void testExternalNestedArrayOutOfBoundAccess() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
- stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE CAST(ARRAYFIELD[10] AS INTEGER) = 200");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(0, values.size());
- }
-
- @Test(expected = ValidationException.class)
- public void testExternalUdfType() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
- stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
- stmt.add("SELECT STREAM MYPLUS(NAME, 1) FROM FOO WHERE ID = 0");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- System.out.println(values);
-
- }
-
- @Test(expected = CompilingClassLoader.CompilerException.class)
- public void testExternalUdfType2() throws Exception {
- List<String> stmt = new ArrayList<>();
- // generated code will be not compilable since return type of MYPLUS and type of 'x' are different
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
- stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
- stmt.add("SELECT STREAM ID FROM FOO WHERE MYPLUS(ID, 1) = 'x'");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(0, values.size());
- }
-
- @Test
- public void testExternalUdf() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
- stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
- stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(2, values.size());
- Assert.assertEquals(4, values.get(0).get(0));
- Assert.assertEquals(5, values.get(1).get(0));
- }
-
- @Test(expected = UnsupportedOperationException.class)
- public void testExternalUdfUsingJar() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
- stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus' USING JAR 'foo'");
- stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- }
-
- @Test
- public void testGroupbyBuiltin() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(SALARY) FROM FOO GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(4, values.size());
- Assert.assertEquals(3, values.get(0).get(2));
- Assert.assertEquals(12, values.get(1).get(2));
- Assert.assertEquals(21, values.get(2).get(2));
- Assert.assertEquals(9, values.get(3).get(2));
- }
-
- @Test
- public void testGroupbyBuiltinWithFilter() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(1, values.size());
- Assert.assertEquals(1, values.get(0).get(0));
- Assert.assertEquals(3L, values.get(0).get(1));
- Assert.assertEquals(12, values.get(0).get(2));
- Assert.assertEquals(2.5, values.get(0).get(3));
- }
-
- @Test
- public void testGroupbyBuiltinAndUDF() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("CREATE FUNCTION MYCONCAT AS 'org.apache.storm.sql.TestUtils$MyConcat'");
- stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
- stmt.add("SELECT STREAM ID, SUM(SALARY), MYCONCAT(NAME), TOPN(2, SALARY) FROM FOO GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(4, values.size());
- Assert.assertEquals(3, values.get(0).get(1));
- Assert.assertEquals("xxx", values.get(0).get(2));
- Assert.assertEquals(Arrays.asList(2, 1), values.get(0).get(3));
- Assert.assertEquals(12, values.get(1).get(1));
- Assert.assertEquals("xxx", values.get(1).get(2));
- Assert.assertEquals(Arrays.asList(5, 4), values.get(1).get(3));
- Assert.assertEquals(21, values.get(2).get(1));
- Assert.assertEquals("xxx", values.get(2).get(2));
- Assert.assertEquals(Arrays.asList(8, 7), values.get(2).get(3));
- Assert.assertEquals(9, values.get(3).get(1));
- Assert.assertEquals("x", values.get(3).get(2));
- Assert.assertEquals(Arrays.asList(9), values.get(3).get(3));
- }
-
- @Test
- public void testAggFnNonSqlReturnType() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
- stmt.add("SELECT STREAM ID, SUM(SALARY), TOPN(1, SALARY) FROM FOO WHERE ID >= 0 GROUP BY (ID) HAVING MAX(SALARY) > 0");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(4, values.size());
- Assert.assertEquals(Collections.singletonList(2), values.get(0).get(2));
- Assert.assertEquals(Collections.singletonList(5), values.get(1).get(2));
- Assert.assertEquals(Collections.singletonList(8), values.get(2).get(2));
- Assert.assertEquals(Collections.singletonList(9), values.get(3).get(2));
- }
-
- @Test
- public void testGroupbySameAggregateOnDifferentColumns() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, COUNT(*), AVG(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(1, values.size());
- Assert.assertEquals(1, values.get(0).get(0));
- Assert.assertEquals(3L, values.get(0).get(1));
- Assert.assertEquals(4, values.get(0).get(2));
- Assert.assertEquals(2.5, values.get(0).get(3));
- }
-
- @Test(expected = UnsupportedOperationException.class)
- public void testGroupbyBuiltinNotimplemented() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, COUNT(*), STDDEV_POP(SALARY) FROM FOO GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- }
-
- @Test
- public void testMinMax() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY (ID)");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(4, values.size());
- Assert.assertEquals(0, values.get(0).get(2));
- Assert.assertEquals(3, values.get(1).get(2));
- Assert.assertEquals(6, values.get(2).get(2));
- Assert.assertEquals(9, values.get(3).get(2));
-
- Assert.assertEquals(1.5, values.get(0).get(3));
- Assert.assertEquals(3.0, values.get(1).get(3));
- Assert.assertEquals(4.5, values.get(2).get(3));
- Assert.assertEquals(5.0, values.get(3).get(3));
- }
- @Test
- public void testFilterGroupbyHaving() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM ID, MIN(SALARY) FROM FOO where ID > 0 GROUP BY (ID) HAVING ID > 2 AND MAX(SALARY) > 5");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(1, values.size());
- Assert.assertEquals(3, values.get(0).get(0));
- Assert.assertEquals(9, values.get(0).get(1));
- }
-
- @Test
- public void testGroupByMultipleFields() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE FOO (DEPTID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR, EMPID INT) LOCATION 'mockgroup:///foo'");
- stmt.add("SELECT STREAM DEPTID, EMPID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY DEPTID, EMPID");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(7, values.size());
- Assert.assertEquals(0, values.get(0).get(0));
- Assert.assertEquals(0, values.get(0).get(1));
- Assert.assertEquals(2L, values.get(0).get(2));
- }
-
- @Test
- public void testjoin() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
- stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
- stmt.add("SELECT STREAM EMPID, EMPNAME, DEPTNAME FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- System.out.println(values);
- Assert.assertEquals(3, values.size());
- Assert.assertEquals("emp1", values.get(0).get(1));
- Assert.assertEquals("dept1", values.get(0).get(2));
- Assert.assertEquals("emp2", values.get(1).get(1));
- Assert.assertEquals("dept1", values.get(1).get(2));
- Assert.assertEquals("emp3", values.get(2).get(1));
- Assert.assertEquals("dept2", values.get(2).get(2));
- }
-
- @Test
- public void testjoinAndGroupby() throws Exception {
- List<String> stmt = new ArrayList<>();
- stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
- stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
- stmt.add("SELECT STREAM d.DEPTID, count(EMPID) FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0" +
- "GROUP BY d.DEPTID");
- StormSql sql = StormSql.construct();
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- sql.execute(stmt, h);
- Assert.assertEquals(2, values.size());
- Assert.assertEquals(1, values.get(0).get(0));
- Assert.assertEquals(2L, values.get(0).get(1));
- Assert.assertEquals(2, values.get(1).get(0));
- Assert.assertEquals(1L, values.get(1).get(1));
- }
-}
[14/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
deleted file mode 100644
index 634e454..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
+++ /dev/null
@@ -1,410 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler;
-
-import com.google.common.base.Function;
-import org.apache.storm.sql.compiler.backends.standalone.TestCompilerUtils;
-import org.apache.storm.tuple.Values;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class TestExprSemantic {
- private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
- RelDataTypeSystem.DEFAULT);
-
- @Test
- public void testLogicalExpr() throws Exception {
- Values v = testExpr(
- Lists.newArrayList("ID > 0 OR ID < 1", "ID > 0 AND ID < 1",
- "NOT (ID > 0 AND ID < 1)"));
- assertEquals(new Values(true, false, true), v);
- }
-
- @Test
- public void testExpectOperator() throws Exception {
- Values v = testExpr(
- Lists.newArrayList("TRUE IS TRUE", "TRUE IS NOT TRUE",
- "UNKNOWN IS TRUE", "UNKNOWN IS NOT TRUE",
- "TRUE IS FALSE", "UNKNOWN IS NULL",
- "UNKNOWN IS NOT NULL"));
- assertEquals(new Values(true, false, false, true, false, true, false), v);
- }
-
- @Test
- public void testDistinctBetweenLikeSimilarIn() throws Exception {
- Values v = testExpr(
- Lists.newArrayList("TRUE IS DISTINCT FROM TRUE",
- "TRUE IS NOT DISTINCT FROM FALSE", "3 BETWEEN 1 AND 5",
- "10 NOT BETWEEN 1 AND 5", "'hello' LIKE '_e%'",
- "'world' NOT LIKE 'wor%'", "'abc' SIMILAR TO '[a-zA-Z]+[cd]{1}'",
- "'abe' NOT SIMILAR TO '[a-zA-Z]+[cd]{1}'", "'3' IN ('1', '2', '3', '4')",
- "2 NOT IN (1, 3, 5)"));
- assertEquals(new Values(false, false, true, true, true,
- false, true, true, true, true), v);
- }
-
- @Test
- public void testCaseStatement() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "CASE WHEN 'abcd' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
- "WHEN UPPER('abcd') = 'AB' THEN 'b' ELSE {fn CONCAT('abcd', '#')} END",
- "CASE WHEN 'ab' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
- "WHEN UPPER('ab') = 'AB' THEN 'b' ELSE {fn CONCAT('ab', '#')} END",
- "CASE WHEN 'abc' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
- "WHEN UPPER('abc') = 'AB' THEN 'b' ELSE {fn CONCAT('abc', '#')} END"
- )
- );
-
- // TODO: The data type of literal Calcite assigns seems to be out of expectation. Please see below logical plan.
- // LogicalProject(EXPR$0=[CASE(OR(=('abcd', 'a'), =('abcd', 'abc'), =('abcd', 'abcde')), CAST(UPPER('a')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abcd'), CAST('AB'):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), 'b', CAST(||('abcd', '#')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL)], EXPR$1=[CASE(OR(=('ab', 'a'), =('ab', 'abc'), =('ab', 'abcde')), CAST(UPPER('a')):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('ab'), 'AB'), CAST('b'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('ab', '#'))], EXPR$2=[CASE(OR(=('abc', 'a'), =('abc', 'abc'), =('abc', 'abcde')), CAST(UPPER('a')):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abc'), CAST('AB'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), CAST('b'):CHAR(4) C
HARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('abc', '#'))]): rowcount = 1.0, cumulative cost = {2.0 rows, 5.0 cpu, 0.0 io}, id = 5
- // LogicalFilter(condition=[AND(>($0, 0), <($0, 2))]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 4
- // EnumerableTableScan(table=[[FOO]]): rowcount = 1.0, cumulative cost = {0.0 rows, 1.0 cpu, 0.0 io}, id = 3
- // in result, both 'b' and UPPER('a') hence 'A' are having some spaces which is not expected.
- // When we use CASE with actual column (Java String type hence VARCHAR), it seems to work as expected.
- // Please refer trident/TestPlanCompiler#testCaseStatement(), and see below logical plan.
- // LogicalProject(EXPR$0=[CASE(OR(=($1, 'a'), =($1, 'abc'), =($1, 'abcde')), CAST(UPPER('a')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", =(CAST(UPPER($1)):VARCHAR(2) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", 'AB'), 'b', CAST(||($1, '#')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary")]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 3
- List<Object> v2 = Lists.transform(v, new Function<Object, Object>() {
- @Nullable
- @Override
- public String apply(@Nullable Object o) {
- return ((String) o).trim();
- }
- });
- assertArrayEquals(new Values("abcd#", "b", "A").toArray(), v2.toArray());
- }
-
- @Test
- public void testNullIfAndCoalesce() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "NULLIF(5, 5)", "NULLIF(5, 0)", "COALESCE(NULL, NULL, 5, 4, NULL)", "COALESCE(1, 5)"
- ));
- assertEquals(new Values(null, 5, 5, 1), v);
- }
-
- @Test
- public void testCollectionFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "ELEMENT(ARRAY[3])", "CARDINALITY(ARRAY[1, 2, 3, 4, 5])"
- ));
- assertEquals(new Values(3, 5), v);
- }
-
- @Test(expected = RuntimeException.class)
- public void testElementFunctionMoreThanOneValue() throws Exception {
- testExpr(
- Lists.newArrayList(
- "ELEMENT(ARRAY[1, 2, 3])"
- ));
- fail("ELEMENT with array which has multiple elements should throw exception in runtime.");
- }
-
- @Test
- public void testArithmeticWithNull() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "1 + CAST(NULL AS INT)", "CAST(NULL AS INT) + 1", "CAST(NULL AS INT) + CAST(NULL AS INT)", "1 + 2"
- ));
- assertEquals(new Values(null, null, null, 3), v);
- }
-
- @Test
- public void testNotWithNull() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "NOT TRUE", "NOT FALSE", "NOT UNKNOWN"
- ));
- assertEquals(new Values(false, true, null), v);
- }
-
- @Test
- public void testAndWithNull() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "UNKNOWN AND TRUE", "UNKNOWN AND FALSE", "UNKNOWN AND UNKNOWN",
- "TRUE AND TRUE", "TRUE AND FALSE", "TRUE AND UNKNOWN",
- "FALSE AND TRUE", "FALSE AND FALSE", "FALSE AND UNKNOWN"
- ));
- assertEquals(new Values(null, false, null, true, false, null, false,
- false, false), v);
- }
-
- @Test
- public void testAndWithNullable() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "ADDR = 'a' AND NAME = 'a'", "NAME = 'a' AND ADDR = 'a'", "NAME = 'x' AND ADDR = 'a'", "ADDR = 'a' AND NAME = 'x'"
- ));
- assertEquals(new Values(false, false, null, null), v);
- }
-
- @Test
- public void testOrWithNullable() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "ADDR = 'a' OR NAME = 'a'", "NAME = 'a' OR ADDR = 'a' ", "NAME = 'x' OR ADDR = 'a' ", "ADDR = 'a' OR NAME = 'x'"
- ));
- assertEquals(new Values(null, null, true, true), v);
- }
-
- @Test
- public void testOrWithNull() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "UNKNOWN OR TRUE", "UNKNOWN OR FALSE", "UNKNOWN OR UNKNOWN",
- "TRUE OR TRUE", "TRUE OR FALSE", "TRUE OR UNKNOWN",
- "FALSE OR TRUE", "FALSE OR FALSE", "FALSE OR UNKNOWN"
- ));
- assertEquals(new Values(true, null, null, true, true, true, true,
- false, null), v);
- }
-
- @Test
- public void testEquals() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "1 = 2", "UNKNOWN = UNKNOWN", "'a' = 'a'", "'a' = UNKNOWN", "UNKNOWN = 'a'", "'a' = 'b'",
- "1 <> 2", "UNKNOWN <> UNKNOWN", "'a' <> 'a'", "'a' <> UNKNOWN", "UNKNOWN <> 'a'", "'a' <> 'b'"
- ));
- assertEquals(new Values(false, null, true, null, null, false,
- true, null, false, null, null, true), v);
- }
-
- @Test
- public void testArithmeticFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "POWER(3, 2)", "ABS(-10)", "MOD(10, 3)", "MOD(-10, 3)",
- "CEIL(123.45)", "FLOOR(123.45)"
- ));
-
- assertEquals(new Values(9.0d, 10, 1, -1, new BigDecimal(124), new BigDecimal(123)), v);
-
- // Belows are floating numbers so comparing this with literal is tend to be failing...
- // Picking int value and compare
- Values v2 = testExpr(
- Lists.newArrayList(
- "SQRT(255)", "LN(16)", "LOG10(10000)", "EXP(10)"
- ));
- List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
- @Nullable
- @Override
- public Object apply(@Nullable Object o) {
- // only takes int value
- return ((Number) o).intValue();
- }
- });
-
- // 15.9687, 2.7725, 4.0, 22026.465794
- assertEquals(new Values(15, 2, 4, 22026), v2m);
- }
-
- @Test
- public void testStringFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "'ab' || 'cd'", "CHAR_LENGTH('foo')", "CHARACTER_LENGTH('foo')",
- "UPPER('a')", "LOWER('A')", "POSITION('bc' IN 'abcd')",
- "TRIM(BOTH ' ' FROM ' abcdeabcdeabc ')",
- "TRIM(LEADING ' ' FROM ' abcdeabcdeabc ')",
- "TRIM(TRAILING ' ' FROM ' abcdeabcdeabc ')",
- "OVERLAY('abcde' PLACING 'bc' FROM 3)",
- "SUBSTRING('abcde' FROM 3)", "SUBSTRING('abcdeabcde' FROM 3 FOR 4)",
- "INITCAP('foo')"
- ));
- assertEquals(new Values("abcd", 3, 3, "A", "a", 2, "abcdeabcdeabc", "abcdeabcdeabc ", " abcdeabcdeabc", "abbce", "cde", "cdea", "Foo"), v);
- }
-
- @Test
- public void testBinaryStringFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "x'45F0AB' || x'45F0AB'",
- "POSITION(x'F0' IN x'453423F0ABBC')",
- "OVERLAY(x'453423F0ABBC45' PLACING x'4534' FROM 3)"
- // "SUBSTRING(x'453423F0ABBC' FROM 3)",
- // "SUBSTRING(x'453423F0ABBC453423F0ABBC' FROM 3 FOR 4)"
- ));
-
- // TODO: Calcite 1.9.0 has bugs on binary SUBSTRING functions
- // as there's no SqlFunctions.substring(org.apache.calcite.avatica.util.ByteString, ...)
- // commented out testing substring function
-
- assertEquals("45f0ab45f0ab", v.get(0).toString());
- assertEquals(4, v.get(1));
- assertEquals("45344534abbc45", v.get(2).toString());
- // assertEquals("23f0abbc", v.get(3).toString());
- // assertEquals("23f0ab", v.get(4).toString());
- }
-
- @Test
- public void testDateAndTimestampLiteral() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "DATE '1970-05-15' AS datefield",
- "TIME '00:00:00' AS timefield",
- "TIMESTAMP '2016-01-01 00:00:00' as timestampfield"
- )
- );
-
- assertEquals(3, v.size());
- assertEquals(134, v.get(0));
- assertEquals(0, v.get(1));
- assertEquals(1451606400000L, v.get(2));
- }
-
- @Test
- public void testInterval() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "INTERVAL '1-5' YEAR TO MONTH AS intervalfield",
- "(DATE '1970-01-01', DATE '1970-01-15') AS anchoredinterval_field"
- )
- );
-
- assertEquals(3, v.size());
- assertEquals(17, v.get(0));
- assertEquals(0, v.get(1));
- assertEquals(14, v.get(2));
- }
-
- @Test
- public void testDateFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "LOCALTIME = CURRENT_TIME, LOCALTIMESTAMP = CURRENT_TIMESTAMP, CURRENT_DATE",
- "EXTRACT(MONTH FROM TIMESTAMP '2010-01-23 12:34:56')",
- "FLOOR(DATE '2016-01-23' TO MONTH)",
- "CEIL(TIME '12:34:56' TO MINUTE)"
- )
- );
-
- assertEquals(6, v.size());
- assertTrue((boolean) v.get(0));
- assertTrue((boolean) v.get(1));
- // skip checking CURRENT_DATE since we don't inject dataContext so don't know about current timestamp
- // we can do it from trident test
- assertEquals(1L, v.get(3));
- assertEquals(0L, v.get(4));
- assertEquals(45300000, v.get(5));
- }
-
- @Test
- public void testJDBCNumericFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "{fn POWER(3, 2)}", "{fn ABS(-10)}", "{fn MOD(10, 3)}", "{fn MOD(-10, 3)}"
- ));
-
- assertEquals(new Values(9.0d, 10, 1, -1), v);
-
- // Belows are floating numbers so comparing this with literal is tend to be failing...
- // Picking int value and compare
- Values v2 = testExpr(
- Lists.newArrayList(
- "{fn LOG(16)}", "{fn LOG10(10000)}", "{fn EXP(10)}"
- ));
- List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
- @Nullable
- @Override
- public Object apply(@Nullable Object o) {
- // only takes int value
- return ((Number) o).intValue();
- }
- });
-
- // 2.7725, 4.0, 22026.465794
- assertEquals(new Values(2, 4, 22026), v2m);
- }
-
- @Test
- public void testJDBCStringFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "{fn CONCAT('ab', 'cd')}",
- "{fn LOCATE('bc', 'abcdeabcde')}",
- //"{fn LOCATE('bc', 'abcdeabcde', 4)}",
- "{fn INSERT('abcd', 2, 3, 'de')}",
- "{fn LCASE('AbCdE')}",
- "{fn LENGTH('AbCdE')}",
- //"{fn LTRIM(' abcde ')}",
- //"{fn RTRIM(' abcde ')}",
- "{fn SUBSTRING('abcdeabcde', 3, 4)}",
- "{fn UCASE('AbCdE')}"
- )
- );
-
- // TODO: Calcite 1.9.0 doesn't support {fn LOCATE(string1, string2 [, integer])}
- // while it's on support list on SQL reference
- // and bugs on LTRIM and RTRIM : throwing AssertionError: Internal error: pre-condition failed: pos != null
- // commented out problematic function tests
-
- assertEquals(new Values("abcd", 2, "ade", "abcde", 5, "cdea", "ABCDE"), v);
- }
-
- @Test
- public void testJDBCDateTimeFunctions() throws Exception {
- Values v = testExpr(
- Lists.newArrayList(
- "{fn CURDATE()} = CURRENT_DATE", "{fn CURTIME()} = LOCALTIME", "{fn NOW()} = LOCALTIMESTAMP",
- "{fn QUARTER(DATE '2016-10-07')}", "{fn TIMESTAMPADD(MINUTE, 15, TIMESTAMP '2016-10-07 00:00:00')}",
- "{fn TIMESTAMPDIFF(SECOND, TIMESTAMP '2016-10-06 00:00:00', TIMESTAMP '2016-10-07 00:00:00')}"
- )
- );
-
- assertEquals(new Values(true, true, true, 4L, 1475799300000L, 86400), v);
- }
-
- private Values testExpr(List<String> exprs) throws Exception {
- String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
- " WHERE ID > 0 AND ID < 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(state.tree());
- Map<String, DataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockDataSource());
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- proc.initialize(data, h);
- return values.get(0);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
deleted file mode 100644
index 8e64e9c..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.StreamableTable;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.compiler.CompilerUtil;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class TestCompilerUtils {
-
- public static class MyPlus {
- public static Integer eval(Integer x, Integer y) {
- return x + y;
- }
- }
-
- public static class MyStaticSumFunction {
- public static long init() {
- return 0L;
- }
- public static long add(long accumulator, int v) {
- return accumulator + v;
- }
- }
-
- public static class MySumFunction {
- public MySumFunction() {
- }
- public long init() {
- return 0L;
- }
- public long add(long accumulator, int v) {
- return accumulator + v;
- }
- public long result(long accumulator) {
- return accumulator;
- }
- }
-
- public static CalciteState sqlOverDummyTable(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("ID", SqlTypeName.INTEGER)
- .field("NAME", typeFactory.createType(String.class))
- .field("ADDR", typeFactory.createType(String.class))
- .build();
- Table table = streamableTable.stream();
- schema.add("FOO", table);
- schema.add("BAR", table);
- schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
- List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
- sqlOperatorTables.add(SqlStdOperatorTable.instance());
- sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
- false,
- Collections.<String>emptyList(), typeFactory));
- SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
- FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
- schema).operatorTable(chainedSqlOperatorTable).build();
- Planner planner = Frameworks.getPlanner(config);
- SqlNode parse = planner.parse(sql);
- SqlNode validate = planner.validate(parse);
- RelNode tree = planner.convert(validate);
- System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static CalciteState sqlOverNestedTable(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
-
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("ID", SqlTypeName.INTEGER)
- .field("MAPFIELD",
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true))
- , true))
- .field("NESTEDMAPFIELD",
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true))
- , true))
- , true))
- .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
- typeFactory.createArrayType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
- , true))
- .build();
- Table table = streamableTable.stream();
- schema.add("FOO", table);
- schema.add("BAR", table);
- schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
- List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
- sqlOperatorTables.add(SqlStdOperatorTable.instance());
- sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
- false,
- Collections.<String>emptyList(), typeFactory));
- SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
- FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
- schema).operatorTable(chainedSqlOperatorTable).build();
- Planner planner = Frameworks.getPlanner(config);
- SqlNode parse = planner.parse(sql);
- SqlNode validate = planner.validate(parse);
- RelNode tree = planner.convert(validate);
- System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static class CalciteState {
- final SchemaPlus schema;
- final RelNode tree;
-
- private CalciteState(SchemaPlus schema, RelNode tree) {
- this.schema = schema;
- this.tree = tree;
- }
-
- public SchemaPlus schema() {
- return schema;
- }
-
- public RelNode tree() {
- return tree;
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
deleted file mode 100644
index 3226810..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.storm.tuple.Values;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TestPlanCompiler {
- private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
- RelDataTypeSystem.DEFAULT);
-
- @Test
- public void testCompile() throws Exception {
- String sql = "SELECT ID + 1 FROM FOO WHERE ID > 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(state.tree());
- Map<String, DataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockDataSource());
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- proc.initialize(data, h);
- Assert.assertArrayEquals(new Values[] { new Values(4), new Values(5)},
- values.toArray());
- }
-
- @Test
- public void testLogicalExpr() throws Exception {
- String sql = "SELECT ID > 0 OR ID < 1, ID > 0 AND ID < 1, NOT (ID > 0 AND ID < 1) FROM FOO WHERE ID > 0 AND ID < 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(state.tree());
- Map<String, DataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockDataSource());
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- proc.initialize(data, h);
- Assert.assertEquals(new Values(true, false, true), values.get(0));
- }
-
- @Test
- public void testNested() throws Exception {
- String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(state.tree());
- Map<String, DataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockNestedDataSource());
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- proc.initialize(data, h);
- Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
- Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
- Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
- }
-
- @Test
- public void testUdf() throws Exception {
- String sql = "SELECT MYPLUS(ID, 3)" +
- "FROM FOO " +
- "WHERE ID = 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(state.tree());
- Map<String, DataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockDataSource());
- List<Values> values = new ArrayList<>();
- ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
- proc.initialize(data, h);
- Assert.assertEquals(new Values(5), values.get(0));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
deleted file mode 100644
index 4bee9aa..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.Collections;
-
-import static org.hamcrest.CoreMatchers.containsString;
-
-public class TestRelNodeCompiler {
- @Test
- public void testFilter() throws Exception {
- String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
- RelDataTypeSystem.DEFAULT);
- LogicalProject project = (LogicalProject) state.tree();
- LogicalFilter filter = (LogicalFilter) project.getInput();
-
- try (StringWriter sw = new StringWriter();
- PrintWriter pw = new PrintWriter(sw)
- ) {
- RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
- // standalone mode doesn't use inputstreams argument
- compiler.visitFilter(filter, Collections.EMPTY_LIST);
- pw.flush();
- Assert.assertThat(sw.toString(), containsString("> 3"));
- }
-
- try (StringWriter sw = new StringWriter();
- PrintWriter pw = new PrintWriter(sw)
- ) {
- RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
- // standalone mode doesn't use inputstreams argument
- compiler.visitProject(project, Collections.EMPTY_LIST);
- pw.flush();
- Assert.assertThat(sw.toString(), containsString(" + 1"));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
deleted file mode 100644
index f6ef1ca..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.trident;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.StreamableTable;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.compiler.CompilerUtil;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.apache.storm.sql.planner.StormRelUtils;
-
-public class TestCompilerUtils {
-
- public static class MyPlus {
- public static Integer eval(Integer x, Integer y) {
- return x + y;
- }
- }
-
- public static class MyStaticSumFunction {
- public static long init() {
- return 0L;
- }
- public static long add(long accumulator, int v) {
- return accumulator + v;
- }
- }
-
- public static class MySumFunction {
- public MySumFunction() {
- }
- public long init() {
- return 0L;
- }
- public long add(long accumulator, int v) {
- return accumulator + v;
- }
- public long result(long accumulator) {
- return accumulator;
- }
- }
-
- public static CalciteState sqlOverDummyTable(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("ID", SqlTypeName.INTEGER)
- .field("NAME", typeFactory.createType(String.class))
- .field("ADDR", typeFactory.createType(String.class))
- .build();
- Table table = streamableTable.stream();
- schema.add("FOO", table);
- schema.add("BAR", table);
- schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
- QueryPlanner queryPlanner = new QueryPlanner(schema);
- TridentRel tree = queryPlanner.getPlan(sql);
- System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static CalciteState sqlOverDummyGroupByTable(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("ID", SqlTypeName.INTEGER)
- .field("GRPID", SqlTypeName.INTEGER)
- .field("NAME", typeFactory.createType(String.class))
- .field("ADDR", typeFactory.createType(String.class))
- .field("AGE", SqlTypeName.INTEGER)
- .field("SCORE", SqlTypeName.INTEGER)
- .build();
- Table table = streamableTable.stream();
- schema.add("FOO", table);
- schema.add("BAR", table);
- schema.add("MYSTATICSUM", AggregateFunctionImpl.create(MyStaticSumFunction.class));
- schema.add("MYSUM", AggregateFunctionImpl.create(MySumFunction.class));
-
- QueryPlanner queryPlanner = new QueryPlanner(schema);
- TridentRel tree = queryPlanner.getPlan(sql);
- System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static CalciteState sqlOverNestedTable(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
-
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("ID", SqlTypeName.INTEGER)
- .field("MAPFIELD",
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true))
- , true))
- .field("NESTEDMAPFIELD",
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createMapType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true))
- , true))
- , true))
- .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
- typeFactory.createArrayType(
- typeFactory.createTypeWithNullability(
- typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
- , true))
- .build();
- Table table = streamableTable.stream();
- schema.add("FOO", table);
- schema.add("BAR", table);
- schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
- QueryPlanner queryPlanner = new QueryPlanner(schema);
- TridentRel tree = queryPlanner.getPlan(sql);
- System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static CalciteState sqlOverSimpleEquiJoinTables(String sql)
- throws RelConversionException, ValidationException, SqlParseException {
- SchemaPlus schema = Frameworks.createRootSchema(true);
- JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
- (RelDataTypeSystem.DEFAULT);
-
- StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("EMPID", SqlTypeName.INTEGER)
- .field("EMPNAME", SqlTypeName.VARCHAR)
- .field("DEPTID", SqlTypeName.INTEGER)
- .build();
- Table table = streamableTable.stream();
-
- StreamableTable streamableTable2 = new CompilerUtil.TableBuilderInfo(typeFactory)
- .field("DEPTID", SqlTypeName.INTEGER)
- .field("DEPTNAME", SqlTypeName.VARCHAR)
- .build();
- Table table2 = streamableTable2.stream();
-
- schema.add("EMP", table);
- schema.add("DEPT", table2);
-
- QueryPlanner queryPlanner = new QueryPlanner(schema);
- TridentRel tree = queryPlanner.getPlan(sql);
- System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
- return new CalciteState(schema, tree);
- }
-
- public static class CalciteState {
- final SchemaPlus schema;
- final RelNode tree;
-
- private CalciteState(SchemaPlus schema, RelNode tree) {
- this.schema = schema;
- this.tree = tree;
- }
-
- public SchemaPlus schema() {
- return schema;
- }
-
- public RelNode tree() {
- return tree;
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
deleted file mode 100644
index 9ba7267..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * *
- * * Licensed to the Apache Software Foundation (ASF) under one
- * * or more contributor license agreements. See the NOTICE file
- * * distributed with this work for additional information
- * * regarding copyright ownership. The ASF licenses this file
- * * to you under the Apache License, Version 2.0 (the
- * * "License"); you may not use this file except in compliance
- * * with the License. You may obtain a copy of the License at
- * * <p>
- * * http://www.apache.org/licenses/LICENSE-2.0
- * * <p>
- * * Unless required by applicable law or agreed to in writing, software
- * * distributed under the License is distributed on an "AS IS" BASIS,
- * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * * See the License for the specific language governing permissions and
- * * limitations under the License.
- *
- */
-package org.apache.storm.sql.compiler.backends.trident;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.AbstractTridentProcessor;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.time.ZoneOffset;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-
-import static org.apache.storm.sql.TestUtils.MockState.getCollectedValues;
-
-public class TestPlanCompiler {
- private static LocalCluster cluster;
-
- @BeforeClass
- public static void staticSetup() throws Exception {
- cluster = new LocalCluster();
- }
-
- @AfterClass
- public static void staticCleanup() {
- if (cluster!= null) {
- cluster.shutdown();
- cluster = null;
- }
- }
-
- @Before
- public void setUp() {
- getCollectedValues().clear();
- }
-
- @Test
- public void testCompile() throws Exception {
- final int EXPECTED_VALUE_SIZE = 2;
- String sql = "SELECT ID FROM FOO WHERE ID > 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- final Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentDataSource());
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final TridentTopology topo = proc.build();
- Fields f = proc.outputStream().getOutputFields();
- proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
- f, new TestUtils.MockStateUpdater(), new Fields());
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
- Assert.assertArrayEquals(new Values[] { new Values(3), new Values(4)}, getCollectedValues().toArray());
- }
-
- @Test
- public void testInsert() throws Exception {
- final int EXPECTED_VALUE_SIZE = 1;
- String sql = "INSERT INTO BAR SELECT ID, NAME, ADDR FROM FOO WHERE ID > 3";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- final Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentDataSource());
- data.put("BAR", new TestUtils.MockSqlTridentDataSource());
-
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final TridentTopology topo = proc.build();
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
- Assert.assertArrayEquals(new Values[] { new Values(4, "abcde", "y")}, getCollectedValues().toArray());
- }
-
- @Test
- public void testUdf() throws Exception {
- int EXPECTED_VALUE_SIZE = 1;
- String sql = "SELECT MYPLUS(ID, 3)" +
- "FROM FOO " +
- "WHERE ID = 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
- Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final TridentTopology topo = proc.build();
- Fields f = proc.outputStream().getOutputFields();
- proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
- f, new TestUtils.MockStateUpdater(), new Fields());
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
- Assert.assertArrayEquals(new Values[] { new Values(5) }, getCollectedValues().toArray());
- }
-
- @Test
- public void testCaseStatement() throws Exception {
- int EXPECTED_VALUE_SIZE = 5;
- String sql = "SELECT CASE WHEN NAME IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
- "WHEN UPPER(NAME) = 'AB' THEN 'b' ELSE {fn CONCAT(NAME, '#')} END FROM FOO";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-
- final Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final TridentTopology topo = proc.build();
- Fields f = proc.outputStream().getOutputFields();
- proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
- Assert.assertArrayEquals(new Values[]{new Values("A"), new Values("b"), new Values("A"), new Values("abcd#"), new Values("A")}, getCollectedValues().toArray());
- }
-
- @Test
- public void testNested() throws Exception {
- int EXPECTED_VALUE_SIZE = 1;
- String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
- "FROM FOO " +
- "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
-
- final Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentNestedDataSource());
-
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final TridentTopology topo = proc.build();
- Fields f = proc.outputStream().getOutputFields();
- proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
- Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
- Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
- Assert.assertArrayEquals(new Values[]{new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300))}, getCollectedValues().toArray());
- }
-
- @Test
- public void testDateKeywords() throws Exception {
- int EXPECTED_VALUE_SIZE = 1;
- String sql = "SELECT " +
- "LOCALTIME, CURRENT_TIME, LOCALTIMESTAMP, CURRENT_TIMESTAMP, CURRENT_DATE " +
- "FROM FOO " +
- "WHERE ID > 0 AND ID < 2";
- TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-
- final Map<String, ISqlTridentDataSource> data = new HashMap<>();
- data.put("FOO", new TestUtils.MockSqlTridentDataSource());
- QueryPlanner planner = new QueryPlanner(state.schema());
- AbstractTridentProcessor proc = planner.compile(data, sql);
- final DataContext dataContext = proc.getDataContext();
- final TridentTopology topo = proc.build();
- Fields f = proc.outputStream().getOutputFields();
- proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
- runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
- long utcTimestamp = (long) dataContext.get(DataContext.Variable.UTC_TIMESTAMP.camelName);
- long currentTimestamp = (long) dataContext.get(DataContext.Variable.CURRENT_TIMESTAMP.camelName);
- long localTimestamp = (long) dataContext.get(DataContext.Variable.LOCAL_TIMESTAMP.camelName);
-
- System.out.println(getCollectedValues());
-
- java.sql.Timestamp timestamp = new java.sql.Timestamp(utcTimestamp);
- int dateInt = (int) timestamp.toLocalDateTime().atOffset(ZoneOffset.UTC).toLocalDate().toEpochDay();
- int localTimeInt = (int) (localTimestamp % DateTimeUtils.MILLIS_PER_DAY);
- int currentTimeInt = (int) (currentTimestamp % DateTimeUtils.MILLIS_PER_DAY);
-
- Assert.assertArrayEquals(new Values[]{new Values(localTimeInt, currentTimeInt, localTimestamp, currentTimestamp, dateInt)}, getCollectedValues().toArray());
- }
-
- private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc,
- TridentTopology topo) throws Exception {
- final Config conf = new Config();
- conf.setMaxSpoutPending(20);
-
- if (proc.getClassLoaders() != null && proc.getClassLoaders().size() > 0) {
- CompilingClassLoader lastClassloader = proc.getClassLoaders().get(proc.getClassLoaders().size() - 1);
- Utils.setClassLoaderForJavaDeSerialize(lastClassloader);
- }
-
- try (LocalTopology stormTopo = cluster.submitTopology("storm-sql", conf, topo.build())) {
- waitForCompletion(1000 * 1000, new Callable<Boolean>() {
- @Override
- public Boolean call() throws Exception {
- return getCollectedValues().size() < expectedValueSize;
- }
- });
- } finally {
- while(cluster.getClusterInfo().get_topologies_size() > 0) {
- Thread.sleep(10);
- }
- Utils.resetClassLoaderForJavaDeSerialize();
- }
- }
-
- private void waitForCompletion(long timeout, Callable<Boolean> cond) throws Exception {
- long start = TestUtils.monotonicNow();
- while (TestUtils.monotonicNow() - start < timeout && cond.call()) {
- Thread.sleep(100);
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
deleted file mode 100644
index 68054d8..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.parser;
-
-import org.apache.calcite.sql.SqlNode;
-import org.apache.storm.sql.parser.impl.ParseException;
-import org.junit.Test;
-
-public class TestSqlParser {
- @Test
- public void testCreateTable() throws Exception {
- String sql = "CREATE EXTERNAL TABLE foo (bar INT) LOCATION 'kafka:///foo'";
- parse(sql);
- }
-
- @Test
- public void testCreateTableWithPrimaryKey() throws Exception {
- String sql = "CREATE EXTERNAL TABLE foo (bar INT PRIMARY KEY ASC) LOCATION 'kafka:///foo'";
- parse(sql);
- }
-
- @Test(expected = ParseException.class)
- public void testCreateTableWithoutLocation() throws Exception {
- String sql = "CREATE EXTERNAL TABLE foo (bar INT)";
- parse(sql);
- }
-
- @Test
- public void testCreateFunction() throws Exception {
- String sql = "CREATE FUNCTION foo AS 'org.apache.storm.sql.MyUDF'";
- parse(sql);
- }
-
- private static SqlNode parse(String sql) throws Exception {
- StormParser parser = new StormParser(sql);
- return parser.impl().parseSqlStmtEof();
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml b/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
deleted file mode 100644
index 9e0a599..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
+++ /dev/null
@@ -1,104 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-hdfs</artifactId>
-
- <developers>
- <developer>
- <id>vesense</id>
- <name>Xin Wang</name>
- <email>data.xinwang@gmail.com</email>
- </developer>
- </developers>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- <exclusions>
- <!--log4j-over-slf4j must be excluded for hadoop-minicluster
- see: http://stackoverflow.com/q/20469026/3542091 -->
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>log4j-over-slf4j</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-hdfs</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-minicluster</artifactId>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
- </exclusion>
- </exclusions>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/resources</directory>
- </resource>
- </resources>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
deleted file mode 100644
index 38c3fcb..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.hdfs;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.hdfs.trident.HdfsState;
-import org.apache.storm.hdfs.trident.HdfsStateFactory;
-import org.apache.storm.hdfs.trident.HdfsUpdater;
-import org.apache.storm.hdfs.trident.format.FileNameFormat;
-import org.apache.storm.hdfs.trident.format.RecordFormat;
-import org.apache.storm.hdfs.trident.format.SimpleFileNameFormat;
-import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
-import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
-import org.apache.storm.hdfs.trident.rotation.TimedRotationPolicy;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a HDFS sink based on the URI and properties. The URI has the format of hdfs://host:port/path-to-file
- * The properties are in JSON format which specifies the name / path of the hdfs file and etc.
- */
-public class HdfsDataSourcesProvider implements DataSourcesProvider {
-
- private static class HdfsTridentDataSource implements ISqlTridentDataSource {
- private final String url;
- private final Properties props;
- private final IOutputSerializer serializer;
-
- private HdfsTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
- this.url = url;
- this.props = props;
- this.serializer = serializer;
- }
-
- @Override
- public ITridentDataSource getProducer() {
- throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
- }
-
- @Override
- public SqlTridentConsumer getConsumer() {
- FileNameFormat fileNameFormat = new SimpleFileNameFormat()
- .withPath(props.getProperty("hdfs.file.path", "/storm"))
- .withName(props.getProperty("hdfs.file.name", "$TIME.$NUM.txt"));
-
- RecordFormat recordFormat = new TridentRecordFormat(serializer);
-
- FileRotationPolicy rotationPolicy;
- String size = props.getProperty("hdfs.rotation.size.kb");
- String interval = props.getProperty("hdfs.rotation.time.seconds");
- Preconditions.checkArgument(size != null || interval != null, "Hdfs data source must contain file rotation config");
-
- if (size != null) {
- rotationPolicy = new FileSizeRotationPolicy(Float.parseFloat(size), FileSizeRotationPolicy.Units.KB);
- } else {
- rotationPolicy = new TimedRotationPolicy(Float.parseFloat(interval), TimedRotationPolicy.TimeUnit.SECONDS);
- }
-
- HdfsState.Options options = new HdfsState.HdfsFileOptions()
- .withFileNameFormat(fileNameFormat)
- .withRecordFormat(recordFormat)
- .withRotationPolicy(rotationPolicy)
- .withFsUrl(url);
-
- StateFactory stateFactory = new HdfsStateFactory().withOptions(options);
- StateUpdater stateUpdater = new HdfsUpdater();
-
- return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
- }
- }
-
- private static class TridentRecordFormat implements RecordFormat {
- private final IOutputSerializer serializer;
-
- private TridentRecordFormat(IOutputSerializer serializer) {
- this.serializer = serializer;
- }
-
- @Override
- public byte[] format(TridentTuple tuple) {
- //TODO we should handle '\n'. ref DelimitedRecordFormat
- return serializer.write(tuple.getValues(), null).array();
- }
-
- }
-
- @Override
- public String scheme() {
- return "hdfs";
- }
-
- @Override
- public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
- List<FieldInfo> fields) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
- Properties properties, List<FieldInfo> fields) {
- List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
- IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
- return new HdfsTridentDataSource(uri.toString(), properties, serializer);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 5fac84f..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.storm.sql.hdfs.HdfsDataSourcesProvider
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
deleted file mode 100644
index 1473438..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.hdfs;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.storm.hdfs.trident.HdfsState;
-import org.apache.storm.hdfs.trident.HdfsStateFactory;
-import org.apache.storm.hdfs.trident.HdfsUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.storm.hdfs.trident.HdfsState.HdfsFileOptions;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-public class TestHdfsDataSourcesProvider {
- private static final List<FieldInfo> FIELDS = ImmutableList.of(
- new FieldInfo("ID", int.class, true),
- new FieldInfo("val", String.class, false));
- private static final Properties TBL_PROPERTIES = new Properties();
-
- private static String hdfsURI;
- private static MiniDFSCluster hdfsCluster;
-
- static {
- TBL_PROPERTIES.put("hdfs.file.path", "/unittest");
- TBL_PROPERTIES.put("hdfs.file.name", "test1.txt");
- TBL_PROPERTIES.put("hdfs.rotation.time.seconds", "120");
- }
-
- @Before
- public void setup() throws Exception {
- Configuration conf = new Configuration();
- conf.set("fs.trash.interval", "10");
- conf.setBoolean("dfs.permissions", true);
- File baseDir = new File("./target/hdfs/").getAbsoluteFile();
- FileUtil.fullyDelete(baseDir);
- conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
-
- MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
- hdfsCluster = builder.build();
- hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
- }
-
- @After
- public void shutDown() throws IOException {
- hdfsCluster.shutdown();
- }
-
- @SuppressWarnings("unchecked")
- @Test
- public void testHdfsSink() {
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
- URI.create(hdfsURI), null, null, TBL_PROPERTIES, FIELDS);
- Assert.assertNotNull(ds);
-
- ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
- Assert.assertEquals(HdfsStateFactory.class, consumer.getStateFactory().getClass());
- Assert.assertEquals(HdfsUpdater.class, consumer.getStateUpdater().getClass());
-
- HdfsState state = (HdfsState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
- StateUpdater stateUpdater = consumer.getStateUpdater();
-
- HdfsFileOptions options = mock(HdfsFileOptions.class);
- Whitebox.setInternalState(state, "options", options);
-
- List<TridentTuple> tupleList = mockTupleList();
-
- for (TridentTuple t : tupleList) {
- stateUpdater.updateState(state, Collections.singletonList(t), null);
- try {
- verify(options).execute(Collections.singletonList(t));
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- }
-
- private static List<TridentTuple> mockTupleList() {
- List<TridentTuple> tupleList = new ArrayList<>();
- TridentTuple t0 = mock(TridentTuple.class);
- TridentTuple t1 = mock(TridentTuple.class);
- doReturn(1).when(t0).get(0);
- doReturn(2).when(t1).get(0);
- doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
- doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
- tupleList.add(t0);
- tupleList.add(t1);
- return tupleList;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/pom.xml b/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
deleted file mode 100644
index a7bdf7a..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
+++ /dev/null
@@ -1,93 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-kafka</artifactId>
-
- <developers>
- <developer>
- <id>haohui</id>
- <name>Haohui Mai</name>
- <email>ricetons@gmail.com</email>
- </developer>
- </developers>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-kafka</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>${storm.kafka.artifact.id}</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka-clients</artifactId>
- <version>${storm.kafka.version}</version>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/resources</directory>
- </resource>
- </resources>
- </build>
-</project>
[02/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
new file mode 100644
index 0000000..3b5eedd
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public class Channels {
+ private static final ChannelContext VOID_CTX = new ChannelContext() {
+ @Override
+ public void emit(Values data) {}
+
+ @Override
+ public void fireChannelInactive() {}
+
+ @Override
+ public void flush() {
+
+ }
+
+ @Override
+ public void setSource(java.lang.Object source) {
+
+ }
+ };
+
+ private static class ChannelContextAdapter implements ChannelContext {
+ private final ChannelHandler handler;
+ private final ChannelContext next;
+
+ public ChannelContextAdapter(
+ ChannelContext next, ChannelHandler handler) {
+ this.handler = handler;
+ this.next = next;
+ }
+
+ @Override
+ public void emit(Values data) {
+ handler.dataReceived(next, data);
+ }
+
+ @Override
+ public void fireChannelInactive() {
+ handler.channelInactive(next);
+ }
+
+ @Override
+ public void flush() {
+ handler.flush(next);
+ }
+
+ @Override
+ public void setSource(java.lang.Object source) {
+ handler.setSource(next, source);
+ next.setSource(source); // propagate through the chain
+ }
+ }
+
+ private static class ForwardingChannelContext implements ChannelContext {
+ private final ChannelContext next;
+
+ public ForwardingChannelContext(ChannelContext next) {
+ this.next = next;
+ }
+
+ @Override
+ public void emit(Values data) {
+ next.emit(data);
+ }
+
+ @Override
+ public void fireChannelInactive() {
+ next.fireChannelInactive();
+ }
+
+ @Override
+ public void flush() {
+ next.flush();
+ }
+
+ @Override
+ public void setSource(Object source) {
+ next.setSource(source);
+ }
+ }
+
+ public static ChannelContext chain(
+ ChannelContext next, ChannelHandler handler) {
+ return new ChannelContextAdapter(next, handler);
+ }
+
+ public static ChannelContext voidContext() {
+ return VOID_CTX;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
new file mode 100644
index 0000000..352af73
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+/**
+ * A DataSource ingests data in StormSQL. It provides a series of tuple to
+ * the downstream {@link ChannelHandler}.
+ *
+ */
+public interface DataSource {
+ void open(ChannelContext ctx);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
new file mode 100644
index 0000000..dbece9c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+public interface DataSourcesProvider {
+ /**
+ * @return the scheme of the data source
+ */
+ String scheme();
+
+ /**
+ * Construct a new data source.
+ * @param uri The URI that specifies the data source. The format of the URI
+ * is fully customizable.
+ * @param inputFormatClass the name of the class that deserializes data.
+ * It is null when unspecified.
+ * @param outputFormatClass the name of the class that serializes data. It
+ * is null when unspecified.
+ * @param fields The name of the fields and the schema of the table.
+ */
+ DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields);
+
+ ISqlTridentDataSource constructTrident(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
new file mode 100644
index 0000000..dfefb01
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.ServiceLoader;
+
+public class DataSourcesRegistry {
+ private static final Logger LOG = LoggerFactory.getLogger(
+ DataSourcesRegistry.class);
+ private static final Map<String, DataSourcesProvider> providers;
+
+ static {
+ providers = new HashMap<>();
+ ServiceLoader<DataSourcesProvider> loader = ServiceLoader.load(
+ DataSourcesProvider.class);
+ for (DataSourcesProvider p : loader) {
+ LOG.info("Registering scheme {} with {}", p.scheme(), p);
+ providers.put(p.scheme(), p);
+ }
+ }
+
+ private DataSourcesRegistry() {
+ }
+
+ public static DataSource construct(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ List<FieldInfo> fields) {
+ DataSourcesProvider provider = providers.get(uri.getScheme());
+ if (provider == null) {
+ return null;
+ }
+
+ return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
+ }
+
+ public static ISqlTridentDataSource constructTridentDataSource(
+ URI uri, String inputFormatClass, String outputFormatClass,
+ Properties properties, List<FieldInfo> fields) {
+ DataSourcesProvider provider = providers.get(uri.getScheme());
+ if (provider == null) {
+ return null;
+ }
+
+ return provider.constructTrident(uri, inputFormatClass, outputFormatClass, properties, fields);
+ }
+
+ /**
+ * Allow unit tests to inject data sources.
+ */
+ public static Map<String, DataSourcesProvider> providerMap() {
+ return providers;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
new file mode 100644
index 0000000..03b030b
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import java.io.Serializable;
+
+/**
+ * Describe each column of the field
+ */
+public class FieldInfo implements Serializable {
+ private final String name;
+ private final Class<?> type;
+ private final boolean isPrimary;
+
+ public FieldInfo(String name, Class<?> type, boolean isPrimary) {
+ this.name = name;
+ this.type = type;
+ this.isPrimary = isPrimary;
+ }
+
+ public String name() {
+ return name;
+ }
+
+ public Class<?> type() {
+ return type;
+ }
+
+ public boolean isPrimary() {
+ return isPrimary;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
new file mode 100644
index 0000000..b6670d9
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public interface IOutputSerializer {
+ /**
+ * Serialize the data to a ByteBuffer. The caller can pass in a ByteBuffer so that the serializer can reuse the
+ * memory.
+ *
+ * @return A ByteBuffer contains the serialized result.
+ */
+ ByteBuffer write(List<Object> data, ByteBuffer buffer);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
new file mode 100644
index 0000000..9eae5ae
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+/**
+ * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
+ */
+public interface ISqlTridentDataSource {
+ /**
+ * SqlTridentConsumer is a data structure containing StateFactory and StateUpdater for consuming tuples with State.
+ *
+ * Please note that StateFactory and StateUpdater should use same class which implements State.
+ *
+ * @see org.apache.storm.trident.state.StateFactory
+ * @see org.apache.storm.trident.state.StateUpdater
+ */
+ interface SqlTridentConsumer {
+ StateFactory getStateFactory();
+ StateUpdater getStateUpdater();
+ }
+
+ /**
+ * Provides instance of ITridentDataSource which can be used as producer in Trident.
+ *
+ * Since ITridentDataSource is a marker interface for Trident Spout interfaces, this method should effectively
+ * return an instance of one of these interfaces (can be changed if Trident API evolves) or descendants:
+ * - IBatchSpout
+ * - ITridentSpout
+ * - IPartitionedTridentSpout
+ * - IOpaquePartitionedTridentSpout
+ *
+ * @see org.apache.storm.trident.spout.ITridentDataSource
+ * @see org.apache.storm.trident.spout.IBatchSpout
+ * @see org.apache.storm.trident.spout.ITridentSpout
+ * @see org.apache.storm.trident.spout.IPartitionedTridentSpout
+ * @see org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout
+ */
+ ITridentDataSource getProducer();
+
+ /**
+ * Provides instance of SqlTridentConsumer which can be used as consumer (State) in Trident.
+ *
+ * @see SqlTridentConsumer
+ */
+ SqlTridentConsumer getConsumer();
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
new file mode 100644
index 0000000..c9abd16
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+public class SimpleSqlTridentConsumer implements ISqlTridentDataSource.SqlTridentConsumer {
+ private final StateFactory stateFactory;
+ private final StateUpdater stateUpdater;
+
+ public SimpleSqlTridentConsumer(StateFactory stateFactory, StateUpdater stateUpdater) {
+ this.stateFactory = stateFactory;
+ this.stateUpdater = stateUpdater;
+ }
+
+ @Override
+ public StateFactory getStateFactory() {
+ return stateFactory;
+ }
+
+ @Override
+ public StateUpdater getStateUpdater() {
+ return stateUpdater;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
new file mode 100644
index 0000000..a373483
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+public class StormSqlFunctions {
+ public static Boolean eq(Object b0, Object b1) {
+ if (b0 == null || b1 == null) {
+ return null;
+ }
+ return b0.equals(b1);
+ }
+
+ public static Boolean ne(Object b0, Object b1) {
+ if (b0 == null || b1 == null) {
+ return null;
+ }
+ return !b0.equals(b1);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
new file mode 100644
index 0000000..e78f354
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.calcite;
+
+import org.apache.calcite.interpreter.Context;
+
+public class DebuggableExecutableExpression implements ExecutableExpression {
+ private ExecutableExpression delegate;
+ private String delegateCode;
+
+ public DebuggableExecutableExpression(ExecutableExpression delegate, String delegateCode) {
+ this.delegate = delegate;
+ this.delegateCode = delegateCode;
+ }
+
+ @Override
+ public Object execute(Context context) {
+ return delegate.execute(context);
+ }
+
+ @Override
+ public void execute(Context context, Object[] results) {
+ delegate.execute(context, results);
+ }
+
+ public String getDelegateCode() {
+ return delegateCode;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
new file mode 100644
index 0000000..8416945
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.calcite;
+
+import org.apache.calcite.interpreter.Context;
+
+import java.io.Serializable;
+
+/**
+ * Compiled executable expression.
+ */
+public interface ExecutableExpression extends Serializable {
+ Object execute(Context context);
+ void execute(Context context, Object[] results);
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
new file mode 100644
index 0000000..4861b43
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.calcite;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.util.Holder;
+
+import java.io.Serializable;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+/**
+ * This is based on SlimDataContext in Calcite, and borrow some from DataContextImpl in Calcite.
+ */
+public class StormDataContext implements DataContext, Serializable {
+ private final ImmutableMap<Object, Object> map;
+
+ public StormDataContext() {
+ // Store the time at which the query started executing. The SQL
+ // standard says that functions such as CURRENT_TIMESTAMP return the
+ // same value throughout the query.
+
+ final Holder<Long> timeHolder = Holder.of(System.currentTimeMillis());
+
+ // Give a hook chance to alter the clock.
+ Hook.CURRENT_TIME.run(timeHolder);
+ final long time = timeHolder.get();
+ final TimeZone timeZone = Calendar.getInstance().getTimeZone();
+ final long localOffset = timeZone.getOffset(time);
+ final long currentOffset = localOffset;
+
+ ImmutableMap.Builder<Object, Object> builder = ImmutableMap.builder();
+ builder.put(Variable.UTC_TIMESTAMP.camelName, time)
+ .put(Variable.CURRENT_TIMESTAMP.camelName, time + currentOffset)
+ .put(Variable.LOCAL_TIMESTAMP.camelName, time + localOffset)
+ .put(Variable.TIME_ZONE.camelName, timeZone);
+ map = builder.build();
+ }
+
+ @Override
+ public SchemaPlus getRootSchema() {
+ return null;
+ }
+
+ @Override
+ public JavaTypeFactory getTypeFactory() {
+ return null;
+ }
+
+ @Override
+ public QueryProvider getQueryProvider() {
+ return null;
+ }
+
+ @Override
+ public Object get(String name) {
+ return map.get(name);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
new file mode 100644
index 0000000..0e65220
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.datasource.socket;
+
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
+import org.apache.storm.sql.runtime.datasource.socket.trident.TridentSocketSpout;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a Socket data source based on the URI and properties. The URI has the format of
+ * socket://[host]:[port]. Both of host and port are mandatory.
+ *
+ * Note that it connects to given host and port, and receive the message if it's used for input source,
+ * and send the message if it's used for output data source.
+ */
+public class SocketDataSourcesProvider implements DataSourcesProvider {
+ @Override
+ public String scheme() {
+ return "socket";
+ }
+
+ private static class SocketTridentDataSource implements ISqlTridentDataSource {
+
+ private final String host;
+ private final int port;
+ private final Scheme scheme;
+ private final IOutputSerializer serializer;
+
+ SocketTridentDataSource(Scheme scheme, IOutputSerializer serializer, String host, int port) {
+ this.scheme = scheme;
+ this.serializer = serializer;
+ this.host = host;
+ this.port = port;
+ }
+
+ @Override
+ public ITridentDataSource getProducer() {
+ return new TridentSocketSpout(scheme, host, port);
+ }
+
+ @Override
+ public SqlTridentConsumer getConsumer() {
+ StateFactory stateFactory = new SocketState.Factory(host, port);
+ StateUpdater<SocketState> stateUpdater = new SocketStateUpdater(serializer);
+ return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+ }
+ }
+
+ @Override
+ public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) {
+ String host = uri.getHost();
+ int port = uri.getPort();
+ if (port == -1) {
+ throw new RuntimeException("Port information is not available. URI: " + uri);
+ }
+
+ List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+ Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
+ IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+
+ return new SocketTridentDataSource(scheme, serializer, host, port);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
new file mode 100644
index 0000000..3f85756
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.datasource.socket.trident;
+
+import org.apache.storm.task.IMetricsContext;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.state.StateFactory;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.net.Socket;
+import java.util.Map;
+
+/**
+ * Trident State implementation of Socket. It only supports writing.
+ */
+public class SocketState implements State {
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void beginCommit(Long txid) {
+ // no-op
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void commit(Long txid) {
+ // no-op
+ }
+
+ public static class Factory implements StateFactory {
+ private final String host;
+ private final int port;
+
+ public Factory(String host, int port) {
+ this.host = host;
+ this.port = port;
+ }
+
+ @Override
+ public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+ BufferedWriter out;
+ try {
+ Socket socket = new Socket(host, port);
+ out = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream()));
+ } catch (IOException e) {
+ throw new RuntimeException("Exception while initializing socket for State. host " +
+ host + " port " + port, e);
+ }
+
+ // State doesn't have close() and Storm actually doesn't guarantee so we can't release socket resource anyway
+ return new SocketState(out);
+ }
+ }
+
+ private BufferedWriter out;
+
+ private SocketState(BufferedWriter out) {
+ this.out = out;
+ }
+
+ public void write(String str) throws IOException {
+ out.write(str);
+ }
+
+ public void flush() throws IOException {
+ out.flush();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
new file mode 100644
index 0000000..3062a90
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.datasource.socket.trident;
+
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.state.BaseStateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * StateUpdater for SocketState. Serializes tuple one by one and writes to socket, and finally flushes.
+ */
+public class SocketStateUpdater extends BaseStateUpdater<SocketState> {
+ private static final Logger LOG = LoggerFactory.getLogger(SocketStateUpdater.class);
+
+ private final IOutputSerializer outputSerializer;
+
+ public SocketStateUpdater(IOutputSerializer outputSerializer) {
+ this.outputSerializer = outputSerializer;
+ }
+
+ @Override
+ public void updateState(SocketState state, List<TridentTuple> tuples, TridentCollector collector) {
+ try {
+ for (TridentTuple tuple : tuples) {
+ byte[] array = outputSerializer.write(tuple.getValues(), null).array();
+ String data = new String(array);
+ state.write(data + "\n");
+ }
+ state.flush();
+ } catch (IOException e) {
+ LOG.error("Error while updating state.", e);
+ collector.reportError(e);
+ throw new RuntimeException("Error while updating state.", e);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
new file mode 100644
index 0000000..97f63a7
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.runtime.datasource.socket.trident;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.storm.Config;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.spout.IBatchSpout;
+import org.apache.storm.tuple.Fields;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Trident Spout for Socket data. Only available for Storm SQL, and only use for test purposes.
+ */
+public class TridentSocketSpout implements IBatchSpout {
+ private static final Logger LOG = LoggerFactory.getLogger(TridentSocketSpout.class);
+
+ private final String host;
+ private final int port;
+ private final Scheme scheme;
+
+ private volatile boolean _running = true;
+
+ private BlockingDeque<String> queue;
+ private Socket socket;
+ private Thread readerThread;
+ private BufferedReader in;
+ private ObjectMapper objectMapper;
+
+ private Map<Long, List<List<Object>>> batches;
+
+ public TridentSocketSpout(Scheme scheme, String host, int port) {
+ this.scheme = scheme;
+ this.host = host;
+ this.port = port;
+ }
+
+ @Override
+ public void open(Map conf, TopologyContext context) {
+ this.queue = new LinkedBlockingDeque<>();
+ this.objectMapper = new ObjectMapper();
+ this.batches = new HashMap<>();
+
+ try {
+ socket = new Socket(host, port);
+ in = new BufferedReader(new InputStreamReader(socket.getInputStream()));
+ } catch (IOException e) {
+ throw new RuntimeException("Error opening socket: host " + host + " port " + port);
+ }
+
+ readerThread = new Thread(new SocketReaderRunnable());
+ readerThread.start();
+ }
+
+ @Override
+ public void emitBatch(long batchId, TridentCollector collector) {
+ // read line and parse it to json
+ List<List<Object>> batch = this.batches.get(batchId);
+ if (batch == null) {
+ batch = new ArrayList<>();
+
+ while(queue.peek() != null) {
+ String line = queue.poll();
+ List<Object> values = convertLineToTuple(line);
+ if (values == null) {
+ continue;
+ }
+
+ batch.add(values);
+ }
+
+ this.batches.put(batchId, batch);
+ }
+
+ for (List<Object> list : batch) {
+ collector.emit(list);
+ }
+ }
+
+ private List<Object> convertLineToTuple(String line) {
+ return scheme.deserialize(ByteBuffer.wrap(line.getBytes()));
+ }
+
+ @Override
+ public void ack(long batchId) {
+ this.batches.remove(batchId);
+ }
+
+ @Override
+ public void close() {
+ _running = false;
+ readerThread.interrupt();
+ queue.clear();
+
+ closeQuietly(in);
+ closeQuietly(socket);
+ }
+
+ @Override
+ public Map<String, Object> getComponentConfiguration() {
+ Config conf = new Config();
+ conf.setMaxTaskParallelism(1);
+ return conf;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return scheme.getOutputFields();
+ }
+
+ private class SocketReaderRunnable implements Runnable {
+ public void run() {
+ while (_running) {
+ try {
+ String line = in.readLine();
+ if (line == null) {
+ throw new RuntimeException("EOF reached from the socket. We can't read the data any more.");
+ }
+
+ queue.push(line.trim());
+ } catch (Throwable t) {
+ // This spout is added to test purpose, so just failing fast doesn't hurt much
+ die(t);
+ }
+ }
+ }
+ }
+
+ private void die(Throwable t) {
+ LOG.error("Halting process: TridentSocketSpout died.", t);
+ if (_running || (t instanceof Error)) { //don't exit if not running, unless it is an Error
+ System.exit(11);
+ }
+ }
+
+ private void closeQuietly(final Closeable closeable) {
+ try {
+ if (closeable != null) {
+ closeable.close();
+ }
+ } catch (final IOException ioe) {
+ // ignore
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
new file mode 100644
index 0000000..3bf1a23
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * AvroScheme uses generic(without code generation) instead of specific(with code generation) readers.
+ */
+public class AvroScheme implements Scheme {
+ private final String schemaString;
+ private final List<String> fieldNames;
+ private final CachedSchemas schemas;
+
+ public AvroScheme(String schemaString, List<String> fieldNames) {
+ this.schemaString = schemaString;
+ this.fieldNames = fieldNames;
+ this.schemas = new CachedSchemas();
+ }
+
+ @Override
+ public List<Object> deserialize(ByteBuffer ser) {
+ try {
+ Schema schema = schemas.getSchema(schemaString);
+ DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(Utils.toByteArray(ser), null);
+ GenericRecord record = reader.read(null, decoder);
+
+ ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+ for (String field : fieldNames) {
+ Object value = record.get(field);
+ // Avro strings are stored using a special Avro Utf8 type instead of using Java primitives
+ list.add(SerdeUtils.convertAvroUtf8(value));
+ }
+ return list;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return new Fields(fieldNames);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
new file mode 100644
index 0000000..5dc3393
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.avro;
+
+import com.google.common.base.Preconditions;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * AvroSerializer uses generic(without code generation) instead of specific(with code generation) writers.
+ */
+public class AvroSerializer implements IOutputSerializer, Serializable {
+ private final String schemaString;
+ private final List<String> fieldNames;
+ private final CachedSchemas schemas;
+
+ public AvroSerializer(String schemaString, List<String> fieldNames) {
+ this.schemaString = schemaString;
+ this.fieldNames = fieldNames;
+ this.schemas = new CachedSchemas();
+ }
+
+ @Override
+ public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+ Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schemas");
+ try {
+ Schema schema = schemas.getSchema(schemaString);
+ GenericRecord record = new GenericData.Record(schema);
+ for (int i = 0; i < fieldNames.size(); i++) {
+ record.put(fieldNames.get(i), data.get(i));
+ }
+
+ ByteArrayOutputStream out = new ByteArrayOutputStream();
+ DatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
+ Encoder encoder = EncoderFactory.get().directBinaryEncoder(out, null);
+ writer.write(record, encoder);
+ encoder.flush();
+ byte[] bytes = out.toByteArray();
+ out.close();
+ return ByteBuffer.wrap(bytes);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
new file mode 100644
index 0000000..4f0e747
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.avro;
+
+import org.apache.avro.Schema;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+// TODO this class is reserved for supporting messages with different schemas.
+// current only one schema in the cache
+public class CachedSchemas implements Serializable{
+
+ private final Map<String, Schema> cache = new HashMap<>();
+
+ public Schema getSchema(String schemaString) {
+ Schema schema = cache.get(schemaString);
+ if (schema == null) {
+ schema = new Schema.Parser().parse(schemaString);
+ cache.put(schemaString, schema);
+ }
+ return schema;
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
new file mode 100644
index 0000000..34fb1bb
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.csv;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * CsvScheme uses the standard RFC4180 CSV Parser
+ * One of the difference from Tsv format is that fields with embedded commas will be quoted.
+ * eg: a,"b,c",d is allowed.
+ *
+ * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
+ */
+public class CsvScheme implements Scheme {
+ private final List<String> fieldNames;
+
+ public CsvScheme(List<String> fieldNames) {
+ this.fieldNames = fieldNames;
+ }
+
+ @Override
+ public List<Object> deserialize(ByteBuffer ser) {
+ try {
+ String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
+ CSVParser parser = CSVParser.parse(data, CSVFormat.RFC4180);
+ CSVRecord record = parser.getRecords().get(0);
+ Preconditions.checkArgument(record.size() == fieldNames.size(), "Invalid schema");
+
+ ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+ for (int i = 0; i < record.size(); i++) {
+ list.add(record.get(i));
+ }
+ return list;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return new Fields(fieldNames);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
new file mode 100644
index 0000000..0d3bd74
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.csv;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * CsvSerializer uses the standard RFC4180 CSV Parser
+ * One of the difference from Tsv format is that fields with embedded commas will be quoted.
+ * eg: a,"b,c",d is allowed.
+ *
+ * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
+ */
+public class CsvSerializer implements IOutputSerializer, Serializable {
+ private final List<String> fields; //reserved for future
+
+ public CsvSerializer(List<String> fields) {
+ this.fields = fields;
+ }
+
+ @Override
+ public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+ try {
+ StringWriter writer = new StringWriter();
+ CSVPrinter printer = new CSVPrinter(writer, CSVFormat.RFC4180);
+ for (Object o : data) {
+ printer.print(o);
+ }
+ //since using StringWriter, we do not need to close it.
+ return ByteBuffer.wrap(writer.getBuffer().toString().getBytes(StandardCharsets.UTF_8));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
new file mode 100644
index 0000000..d288fa1
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.json;
+
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+public class JsonScheme implements Scheme {
+ private final List<String> fields;
+
+ public JsonScheme(List<String> fields) {
+ this.fields = fields;
+ }
+
+ @Override
+ public List<Object> deserialize(ByteBuffer ser) {
+ ObjectMapper mapper = new ObjectMapper();
+ try {
+ @SuppressWarnings("unchecked")
+ HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
+ ArrayList<Object> list = new ArrayList<>(fields.size());
+ for (String f : fields) {
+ list.add(map.get(f));
+ }
+ return list;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return new Fields(fields);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
new file mode 100644
index 0000000..1e825c4
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.json;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+public class JsonSerializer implements IOutputSerializer, Serializable {
+ private final List<String> fieldNames;
+ private final JsonFactory jsonFactory;
+
+ public JsonSerializer(List<String> fieldNames) {
+ this.fieldNames = fieldNames;
+ jsonFactory = new JsonFactory();
+ }
+
+ @Override
+ public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+ Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schema");
+ StringWriter sw = new StringWriter();
+ try (JsonGenerator jg = jsonFactory.createGenerator(sw)) {
+ jg.writeStartObject();
+ for (int i = 0; i < fieldNames.size(); ++i) {
+ jg.writeFieldName(fieldNames.get(i));
+ jg.writeObject(data.get(i));
+ }
+ jg.writeEndObject();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
new file mode 100644
index 0000000..310494c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.tsv;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * TsvScheme uses a simple delimited format implemention by splitting string,
+ * and it supports user defined delimiter.
+ */
+public class TsvScheme implements Scheme {
+ private final List<String> fieldNames;
+ private final char delimiter;
+
+ public TsvScheme(List<String> fieldNames, char delimiter) {
+ this.fieldNames = fieldNames;
+ this.delimiter = delimiter;
+ }
+
+ @Override
+ public List<Object> deserialize(ByteBuffer ser) {
+ String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
+ List<String> parts = org.apache.storm.sql.runtime.utils.Utils.split(data, delimiter);
+ Preconditions.checkArgument(parts.size() == fieldNames.size(), "Invalid schema");
+
+ ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+ list.addAll(parts);
+ return list;
+ }
+
+ @Override
+ public Fields getOutputFields() {
+ return new Fields(fieldNames);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
new file mode 100644
index 0000000..1cf1c76
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.serde.tsv;
+
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * TsvSerializer uses a simple delimited format implemention by splitting string,
+ * and it supports user defined delimiter.
+ */
+public class TsvSerializer implements IOutputSerializer, Serializable {
+ private final List<String> fields; //reserved for future
+ private final char delimiter;
+
+ public TsvSerializer(List<String> fields, char delimiter) {
+ this.fields = fields;
+ this.delimiter = delimiter;
+ }
+
+ @Override
+ public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+ StringBuilder sb = new StringBuilder(512); // 512: for most scenes to avoid inner array resizing
+ for (int i = 0; i < data.size(); i++) {
+ Object o = data.get(i);
+ if (i == 0) {
+ sb.append(o);
+ } else {
+ sb.append(delimiter);
+ sb.append(o);
+ }
+ }
+ return ByteBuffer.wrap(sb.toString().getBytes(StandardCharsets.UTF_8));
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
new file mode 100644
index 0000000..6c76481
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.storm.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.OperationAwareFlatMapFunction;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class EvaluationCalc implements OperationAwareFlatMapFunction {
+ private static final Logger LOG = LoggerFactory.getLogger(EvaluationCalc.class);
+
+ private final ExecutableExpression filterInstance;
+ private final ExecutableExpression projectionInstance;
+ private final Object[] outputValues;
+ private final DataContext dataContext;
+
+ public EvaluationCalc(ExecutableExpression filterInstance, ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
+ this.filterInstance = filterInstance;
+ this.projectionInstance = projectionInstance;
+ this.outputValues = new Object[outputCount];
+ this.dataContext = dataContext;
+ }
+
+ @Override
+ public void prepare(Map conf, TridentOperationContext context) {
+ if (projectionInstance != null && projectionInstance instanceof DebuggableExecutableExpression) {
+ LOG.info("Expression code for projection: \n{}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
+ }
+ if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
+ LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
+ }
+ }
+
+ @Override
+ public void cleanup() {
+
+ }
+
+ @Override
+ public Iterable<Values> execute(TridentTuple input) {
+ Context calciteContext = new StormContext(dataContext);
+ calciteContext.values = input.getValues().toArray();
+
+ if (filterInstance != null) {
+ filterInstance.execute(calciteContext, outputValues);
+ // filtered out
+ if (outputValues[0] == null || !((Boolean) outputValues[0])) {
+ return Collections.emptyList();
+ }
+ }
+
+ if (projectionInstance != null) {
+ projectionInstance.execute(calciteContext, outputValues);
+ return Collections.singletonList(new Values(outputValues));
+ } else {
+ return Collections.singletonList(new Values(input.getValues()));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
new file mode 100644
index 0000000..9314852
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.storm.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.BaseFilter;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class EvaluationFilter extends BaseFilter {
+ private static final Logger LOG = LoggerFactory.getLogger(EvaluationFilter.class);
+
+ private final ExecutableExpression filterInstance;
+ private final DataContext dataContext;
+ private final Object[] outputValues;
+
+ public EvaluationFilter(ExecutableExpression filterInstance, DataContext dataContext) {
+ this.filterInstance = filterInstance;
+ this.dataContext = dataContext;
+ this.outputValues = new Object[1];
+ }
+
+ @Override
+ public void prepare(Map conf, TridentOperationContext context) {
+ if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
+ LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
+ }
+ }
+
+ @Override
+ public boolean isKeep(TridentTuple tuple) {
+ Context calciteContext = new StormContext(dataContext);
+ calciteContext.values = tuple.getValues().toArray();
+ filterInstance.execute(calciteContext, outputValues);
+ return (outputValues[0] != null && (boolean) outputValues[0]);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
new file mode 100644
index 0000000..2608104
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.storm.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.OperationAwareMapFunction;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class EvaluationFunction implements OperationAwareMapFunction {
+ private static final Logger LOG = LoggerFactory.getLogger(EvaluationFunction.class);
+
+ private final ExecutableExpression projectionInstance;
+ private final Object[] outputValues;
+ private final DataContext dataContext;
+
+ public EvaluationFunction(ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
+ this.projectionInstance = projectionInstance;
+ this.outputValues = new Object[outputCount];
+ this.dataContext = dataContext;
+ }
+
+ @Override
+ public void prepare(Map conf, TridentOperationContext context) {
+ if (projectionInstance instanceof DebuggableExecutableExpression) {
+ LOG.info("Expression code: {}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
+ }
+ }
+
+ @Override
+ public void cleanup() {
+
+ }
+
+ @Override
+ public Values execute(TridentTuple input) {
+ Context calciteContext = new StormContext(dataContext);
+ calciteContext.values = input.getValues().toArray();
+ projectionInstance.execute(calciteContext, outputValues);
+ return new Values(outputValues);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
new file mode 100644
index 0000000..4c3a266
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.storm.sql.runtime.trident.functions;
+
+import org.apache.storm.trident.operation.BaseFunction;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+public class ForwardFunction extends BaseFunction {
+ @Override
+ public void execute(TridentTuple tuple, TridentCollector collector) {
+ collector.emit(tuple.getValues());
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
new file mode 100644
index 0000000..efd5d25
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime.utils;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.FieldInfo;
+
+import java.io.Serializable;
+import java.util.List;
+
+public final class FieldInfoUtils {
+
+ public static List<String> getFieldNames(List<FieldInfo> fields) {
+ return Lists.transform(fields, new FieldNameExtractor());
+ }
+
+ private static class FieldNameExtractor implements Function<FieldInfo, String>, Serializable {
+ @Override
+ public String apply(FieldInfo fieldInfo) {
+ return fieldInfo.name();
+ }
+ }
+}
[21/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
STORM-2453 Move non-connectors into the top directory
* move storm-sql, flux, storm-submit-tools to the top directory
* rearrange target directories for them in binary dist.
* create "lib-tools" directory for storing non-storm-core libs
* sql: lib-tools/sql/core and lib-tools/sql/runtime
* storm-submit-tools: lib-tools/submit-tools
* storm-kafka-monitor: lib-tools/storm-kafka-monitor
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/e9d78338
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/e9d78338
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/e9d78338
Branch: refs/heads/master
Commit: e9d783386e6d964df925546beb4db55106e61312
Parents: 28c8711
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Apr 4 17:27:32 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Apr 4 23:04:18 2017 +0900
----------------------------------------------------------------------
bin/storm-kafka-monitor | 2 +-
bin/storm.py | 9 +-
external/flux/README.md | 872 -------------------
external/flux/flux-core/pom.xml | 112 ---
.../main/java/org/apache/storm/flux/Flux.java | 261 ------
.../java/org/apache/storm/flux/FluxBuilder.java | 630 --------------
.../apache/storm/flux/api/TopologySource.java | 39 -
.../org/apache/storm/flux/model/BeanDef.java | 39 -
.../storm/flux/model/BeanListReference.java | 37 -
.../apache/storm/flux/model/BeanReference.java | 39 -
.../org/apache/storm/flux/model/BoltDef.java | 24 -
.../storm/flux/model/ConfigMethodDef.java | 67 --
.../storm/flux/model/ExecutionContext.java | 77 --
.../apache/storm/flux/model/GroupingDef.java | 77 --
.../org/apache/storm/flux/model/IncludeDef.java | 54 --
.../org/apache/storm/flux/model/ObjectDef.java | 95 --
.../apache/storm/flux/model/PropertyDef.java | 58 --
.../org/apache/storm/flux/model/SpoutDef.java | 24 -
.../org/apache/storm/flux/model/StreamDef.java | 64 --
.../apache/storm/flux/model/TopologyDef.java | 216 -----
.../storm/flux/model/TopologySourceDef.java | 36 -
.../org/apache/storm/flux/model/VertexDef.java | 36 -
.../apache/storm/flux/parser/FluxParser.java | 208 -----
.../flux-core/src/main/resources/splash.txt | 9 -
.../org/apache/storm/flux/FluxBuilderTest.java | 31 -
.../org/apache/storm/flux/IntegrationTest.java | 39 -
.../java/org/apache/storm/flux/TCKTest.java | 256 ------
.../multilang/MultilangEnvirontmentTest.java | 89 --
.../apache/storm/flux/test/SimpleTopology.java | 59 --
.../storm/flux/test/SimpleTopologySource.java | 52 --
.../test/SimpleTopologyWithConfigParam.java | 55 --
.../org/apache/storm/flux/test/TestBolt.java | 131 ---
.../storm/flux/test/TridentTopologySource.java | 71 --
.../src/test/resources/configs/bad_hbase.yaml | 98 ---
.../test/resources/configs/bad_shell_test.yaml | 118 ---
.../resources/configs/config-methods-test.yaml | 92 --
.../resources/configs/diamond-topology.yaml | 87 --
.../existing-topology-method-override.yaml | 25 -
.../existing-topology-reflection-config.yaml | 24 -
.../configs/existing-topology-reflection.yaml | 24 -
.../configs/existing-topology-trident.yaml | 24 -
.../resources/configs/existing-topology.yaml | 23 -
.../src/test/resources/configs/hdfs_test.yaml | 97 ---
.../test/resources/configs/include_test.yaml | 25 -
.../configs/invalid-existing-topology.yaml | 33 -
.../src/test/resources/configs/kafka_test.yaml | 126 ---
.../src/test/resources/configs/shell_test.yaml | 118 ---
.../test/resources/configs/simple_hbase.yaml | 120 ---
.../resources/configs/substitution-test.yaml | 106 ---
.../src/test/resources/configs/tck.yaml | 95 --
.../src/test/resources/configs/test.properties | 18 -
.../flux-core/src/test/resources/log4j2.xml | 34 -
external/flux/flux-examples/README.md | 93 --
external/flux/flux-examples/pom.xml | 146 ----
.../flux/examples/StatefulWordCounter.java | 64 --
.../storm/flux/examples/TestPrintBolt.java | 39 -
.../storm/flux/examples/TestWindowBolt.java | 47 -
.../storm/flux/examples/WordCountClient.java | 74 --
.../apache/storm/flux/examples/WordCounter.java | 71 --
.../src/main/resources/hbase_bolt.properties | 18 -
.../src/main/resources/hdfs_bolt.properties | 26 -
.../src/main/resources/kafka_spout.yaml | 136 ---
.../src/main/resources/multilang.yaml | 89 --
.../src/main/resources/simple_hbase.yaml | 92 --
.../src/main/resources/simple_hdfs.yaml | 105 ---
.../resources/simple_stateful_wordcount.yaml | 60 --
.../src/main/resources/simple_windowing.yaml | 69 --
.../src/main/resources/simple_wordcount.yaml | 68 --
external/flux/flux-ui/README.md | 3 -
external/flux/flux-wrappers/pom.xml | 51 --
.../flux/wrappers/bolts/FluxShellBolt.java | 175 ----
.../storm/flux/wrappers/bolts/LogInfoBolt.java | 44 -
.../flux/wrappers/spouts/FluxShellSpout.java | 175 ----
.../main/resources/resources/randomsentence.js | 93 --
.../main/resources/resources/splitsentence.py | 24 -
external/flux/pom.xml | 85 --
external/sql/README.md | 207 -----
external/sql/pom.xml | 47 -
external/sql/storm-sql-core/pom.xml | 279 ------
.../sql/storm-sql-core/src/codegen/config.fmpp | 23 -
.../storm-sql-core/src/codegen/data/Parser.tdd | 80 --
.../src/codegen/includes/license.ftl | 17 -
.../src/codegen/includes/parserImpls.ftl | 113 ---
.../storm/sql/AbstractTridentProcessor.java | 58 --
.../src/jvm/org/apache/storm/sql/StormSql.java | 59 --
.../jvm/org/apache/storm/sql/StormSqlImpl.java | 290 ------
.../org/apache/storm/sql/StormSqlRunner.java | 85 --
.../sql/calcite/ParallelStreamableTable.java | 35 -
.../apache/storm/sql/compiler/CompilerUtil.java | 183 ----
.../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 -----
.../sql/compiler/StormSqlTypeFactoryImpl.java | 51 --
.../standalone/BuiltinAggregateFunctions.java | 238 -----
.../backends/standalone/PlanCompiler.java | 139 ---
.../standalone/PostOrderRelNodeVisitor.java | 132 ---
.../backends/standalone/RelNodeCompiler.java | 484 ----------
.../storm/sql/javac/CompilingClassLoader.java | 225 -----
.../storm/sql/parser/ColumnConstraint.java | 42 -
.../storm/sql/parser/ColumnDefinition.java | 44 -
.../storm/sql/parser/SqlCreateFunction.java | 94 --
.../apache/storm/sql/parser/SqlCreateTable.java | 166 ----
.../apache/storm/sql/parser/SqlDDLKeywords.java | 27 -
.../apache/storm/sql/parser/StormParser.java | 47 -
.../apache/storm/sql/parser/UnparseUtil.java | 60 --
.../sql/planner/StormRelDataTypeSystem.java | 37 -
.../apache/storm/sql/planner/StormRelUtils.java | 68 --
.../planner/UnsupportedOperatorsVisitor.java | 24 -
.../storm/sql/planner/rel/StormCalcRelBase.java | 32 -
.../sql/planner/rel/StormFilterRelBase.java | 32 -
.../storm/sql/planner/rel/StormJoinRelBase.java | 36 -
.../sql/planner/rel/StormProjectRelBase.java | 35 -
.../storm/sql/planner/rel/StormRelNode.java | 24 -
.../planner/rel/StormStreamInsertRelBase.java | 36 -
.../sql/planner/rel/StormStreamScanRelBase.java | 34 -
.../storm/sql/planner/trident/QueryPlanner.java | 156 ----
.../sql/planner/trident/TridentPlanCreator.java | 125 ---
.../planner/trident/TridentStormRuleSets.java | 110 ---
.../sql/planner/trident/rel/TridentCalcRel.java | 97 ---
.../planner/trident/rel/TridentFilterRel.java | 65 --
.../trident/rel/TridentLogicalConvention.java | 67 --
.../planner/trident/rel/TridentProjectRel.java | 69 --
.../sql/planner/trident/rel/TridentRel.java | 26 -
.../trident/rel/TridentStreamInsertRel.java | 76 --
.../trident/rel/TridentStreamScanRel.java | 55 --
.../trident/rules/TridentAggregateRule.java | 39 -
.../planner/trident/rules/TridentCalcRule.java | 45 -
.../trident/rules/TridentFilterRule.java | 45 -
.../planner/trident/rules/TridentJoinRule.java | 37 -
.../trident/rules/TridentModifyRule.java | 71 --
.../trident/rules/TridentProjectRule.java | 45 -
.../planner/trident/rules/TridentScanRule.java | 58 --
.../test/org/apache/storm/sql/TestStormSql.java | 492 -----------
.../storm/sql/compiler/TestExprSemantic.java | 410 ---------
.../backends/standalone/TestCompilerUtils.java | 183 ----
.../backends/standalone/TestPlanCompiler.java | 104 ---
.../standalone/TestRelNodeCompiler.java | 64 --
.../backends/trident/TestCompilerUtils.java | 208 -----
.../backends/trident/TestPlanCompiler.java | 235 -----
.../apache/storm/sql/parser/TestSqlParser.java | 54 --
.../storm-sql-external/storm-sql-hdfs/pom.xml | 104 ---
.../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 ---
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/hdfs/TestHdfsDataSourcesProvider.java | 129 ---
.../storm-sql-external/storm-sql-kafka/pom.xml | 93 --
.../sql/kafka/KafkaDataSourcesProvider.java | 170 ----
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/kafka/TestKafkaDataSourcesProvider.java | 125 ---
.../storm-sql-mongodb/pom.xml | 84 --
.../sql/mongodb/MongoDataSourcesProvider.java | 126 ---
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../mongodb/TestMongoDataSourcesProvider.java | 122 ---
.../storm-sql-external/storm-sql-redis/pom.xml | 76 --
.../sql/redis/RedisDataSourcesProvider.java | 228 -----
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../sql/redis/TestRedisDataSourcesProvider.java | 158 ----
external/sql/storm-sql-runtime/pom.xml | 136 ---
.../calcite/interpreter/StormContext.java | 31 -
.../sql/runtime/AbstractChannelHandler.java | 52 --
.../sql/runtime/AbstractValuesProcessor.java | 46 -
.../storm/sql/runtime/ChannelContext.java | 30 -
.../storm/sql/runtime/ChannelHandler.java | 41 -
.../org/apache/storm/sql/runtime/Channels.java | 109 ---
.../apache/storm/sql/runtime/DataSource.java | 27 -
.../storm/sql/runtime/DataSourcesProvider.java | 47 -
.../storm/sql/runtime/DataSourcesRegistry.java | 76 --
.../org/apache/storm/sql/runtime/FieldInfo.java | 47 -
.../storm/sql/runtime/IOutputSerializer.java | 31 -
.../sql/runtime/ISqlTridentDataSource.java | 65 --
.../sql/runtime/SimpleSqlTridentConsumer.java | 41 -
.../storm/sql/runtime/StormSqlFunctions.java | 34 -
.../calcite/DebuggableExecutableExpression.java | 45 -
.../runtime/calcite/ExecutableExpression.java | 31 -
.../sql/runtime/calcite/StormDataContext.java | 79 --
.../socket/SocketDataSourcesProvider.java | 101 ---
.../datasource/socket/trident/SocketState.java | 89 --
.../socket/trident/SocketStateUpdater.java | 59 --
.../socket/trident/TridentSocketSpout.java | 177 ----
.../sql/runtime/serde/avro/AvroScheme.java | 74 --
.../sql/runtime/serde/avro/AvroSerializer.java | 72 --
.../sql/runtime/serde/avro/CachedSchemas.java | 41 -
.../storm/sql/runtime/serde/csv/CsvScheme.java | 70 --
.../sql/runtime/serde/csv/CsvSerializer.java | 59 --
.../sql/runtime/serde/json/JsonScheme.java | 58 --
.../sql/runtime/serde/json/JsonSerializer.java | 57 --
.../storm/sql/runtime/serde/tsv/TsvScheme.java | 58 --
.../sql/runtime/serde/tsv/TsvSerializer.java | 54 --
.../trident/functions/EvaluationCalc.java | 86 --
.../trident/functions/EvaluationFilter.java | 61 --
.../trident/functions/EvaluationFunction.java | 67 --
.../trident/functions/ForwardFunction.java | 30 -
.../storm/sql/runtime/utils/FieldInfoUtils.java | 39 -
.../storm/sql/runtime/utils/SerdeUtils.java | 123 ---
.../apache/storm/sql/runtime/utils/Utils.java | 55 --
...apache.storm.sql.runtime.DataSourcesProvider | 16 -
.../apache/storm/sql/TestAvroSerializer.java | 72 --
.../org/apache/storm/sql/TestCsvSerializer.java | 54 --
.../storm/sql/TestJsonRepresentation.java | 52 --
.../org/apache/storm/sql/TestTsvSerializer.java | 46 -
.../test/org/apache/storm/sql/TestUtils.java | 584 -------------
.../socket/TestSocketDataSourceProvider.java | 94 --
external/storm-submit-tools/pom.xml | 216 -----
.../submit/command/DependencyResolverMain.java | 158 ----
.../storm/submit/dependency/AetherUtils.java | 91 --
.../apache/storm/submit/dependency/Booter.java | 51 --
.../submit/dependency/DependencyResolver.java | 98 ---
.../dependency/RepositorySystemFactory.java | 67 --
.../submit/dependency/AetherUtilsTest.java | 102 ---
.../dependency/DependencyResolverTest.java | 83 --
flux/README.md | 872 +++++++++++++++++++
flux/flux-core/pom.xml | 112 +++
.../main/java/org/apache/storm/flux/Flux.java | 259 ++++++
.../java/org/apache/storm/flux/FluxBuilder.java | 630 ++++++++++++++
.../apache/storm/flux/api/TopologySource.java | 39 +
.../org/apache/storm/flux/model/BeanDef.java | 34 +
.../storm/flux/model/BeanListReference.java | 37 +
.../apache/storm/flux/model/BeanReference.java | 39 +
.../org/apache/storm/flux/model/BoltDef.java | 24 +
.../storm/flux/model/ConfigMethodDef.java | 67 ++
.../storm/flux/model/ExecutionContext.java | 77 ++
.../apache/storm/flux/model/GroupingDef.java | 77 ++
.../org/apache/storm/flux/model/IncludeDef.java | 54 ++
.../org/apache/storm/flux/model/ObjectDef.java | 95 ++
.../apache/storm/flux/model/PropertyDef.java | 58 ++
.../org/apache/storm/flux/model/SpoutDef.java | 24 +
.../org/apache/storm/flux/model/StreamDef.java | 64 ++
.../apache/storm/flux/model/TopologyDef.java | 216 +++++
.../storm/flux/model/TopologySourceDef.java | 36 +
.../org/apache/storm/flux/model/VertexDef.java | 36 +
.../apache/storm/flux/parser/FluxParser.java | 206 +++++
flux/flux-core/src/main/resources/splash.txt | 9 +
.../org/apache/storm/flux/FluxBuilderTest.java | 31 +
.../org/apache/storm/flux/IntegrationTest.java | 39 +
.../java/org/apache/storm/flux/TCKTest.java | 254 ++++++
.../multilang/MultilangEnvirontmentTest.java | 89 ++
.../apache/storm/flux/test/SimpleTopology.java | 58 ++
.../storm/flux/test/SimpleTopologySource.java | 52 ++
.../test/SimpleTopologyWithConfigParam.java | 53 ++
.../org/apache/storm/flux/test/TestBolt.java | 131 +++
.../storm/flux/test/TridentTopologySource.java | 71 ++
.../src/test/resources/configs/bad_hbase.yaml | 98 +++
.../test/resources/configs/bad_shell_test.yaml | 118 +++
.../resources/configs/config-methods-test.yaml | 92 ++
.../resources/configs/diamond-topology.yaml | 87 ++
.../existing-topology-method-override.yaml | 25 +
.../existing-topology-reflection-config.yaml | 24 +
.../configs/existing-topology-reflection.yaml | 24 +
.../configs/existing-topology-trident.yaml | 24 +
.../resources/configs/existing-topology.yaml | 23 +
.../src/test/resources/configs/hdfs_test.yaml | 97 +++
.../test/resources/configs/include_test.yaml | 25 +
.../configs/invalid-existing-topology.yaml | 33 +
.../src/test/resources/configs/kafka_test.yaml | 126 +++
.../src/test/resources/configs/shell_test.yaml | 118 +++
.../test/resources/configs/simple_hbase.yaml | 120 +++
.../resources/configs/substitution-test.yaml | 106 +++
.../src/test/resources/configs/tck.yaml | 95 ++
.../src/test/resources/configs/test.properties | 18 +
flux/flux-core/src/test/resources/log4j2.xml | 34 +
flux/flux-examples/README.md | 93 ++
flux/flux-examples/pom.xml | 146 ++++
.../flux/examples/StatefulWordCounter.java | 64 ++
.../storm/flux/examples/TestPrintBolt.java | 39 +
.../storm/flux/examples/TestWindowBolt.java | 47 +
.../storm/flux/examples/WordCountClient.java | 74 ++
.../apache/storm/flux/examples/WordCounter.java | 71 ++
.../src/main/resources/hbase_bolt.properties | 18 +
.../src/main/resources/hdfs_bolt.properties | 26 +
.../src/main/resources/kafka_spout.yaml | 136 +++
.../src/main/resources/multilang.yaml | 89 ++
.../src/main/resources/simple_hbase.yaml | 92 ++
.../src/main/resources/simple_hdfs.yaml | 105 +++
.../resources/simple_stateful_wordcount.yaml | 60 ++
.../src/main/resources/simple_windowing.yaml | 69 ++
.../src/main/resources/simple_wordcount.yaml | 68 ++
flux/flux-ui/README.md | 3 +
flux/flux-wrappers/pom.xml | 51 ++
.../flux/wrappers/bolts/FluxShellBolt.java | 175 ++++
.../storm/flux/wrappers/bolts/LogInfoBolt.java | 44 +
.../flux/wrappers/spouts/FluxShellSpout.java | 175 ++++
.../main/resources/resources/randomsentence.js | 93 ++
.../main/resources/resources/splitsentence.py | 24 +
flux/pom.xml | 85 ++
pom.xml | 6 +-
sql/README.md | 207 +++++
sql/pom.xml | 47 +
sql/storm-sql-core/pom.xml | 279 ++++++
sql/storm-sql-core/src/codegen/config.fmpp | 23 +
sql/storm-sql-core/src/codegen/data/Parser.tdd | 80 ++
.../src/codegen/includes/license.ftl | 17 +
.../src/codegen/includes/parserImpls.ftl | 113 +++
.../storm/sql/AbstractTridentProcessor.java | 56 ++
.../src/jvm/org/apache/storm/sql/StormSql.java | 59 ++
.../jvm/org/apache/storm/sql/StormSqlImpl.java | 290 ++++++
.../org/apache/storm/sql/StormSqlRunner.java | 85 ++
.../sql/calcite/ParallelStreamableTable.java | 35 +
.../apache/storm/sql/compiler/CompilerUtil.java | 183 ++++
.../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 +++++
.../sql/compiler/StormSqlTypeFactoryImpl.java | 51 ++
.../standalone/BuiltinAggregateFunctions.java | 238 +++++
.../backends/standalone/PlanCompiler.java | 139 +++
.../standalone/PostOrderRelNodeVisitor.java | 132 +++
.../backends/standalone/RelNodeCompiler.java | 484 ++++++++++
.../storm/sql/javac/CompilingClassLoader.java | 225 +++++
.../storm/sql/parser/ColumnConstraint.java | 42 +
.../storm/sql/parser/ColumnDefinition.java | 44 +
.../storm/sql/parser/SqlCreateFunction.java | 94 ++
.../apache/storm/sql/parser/SqlCreateTable.java | 166 ++++
.../apache/storm/sql/parser/SqlDDLKeywords.java | 27 +
.../apache/storm/sql/parser/StormParser.java | 47 +
.../apache/storm/sql/parser/UnparseUtil.java | 59 ++
.../sql/planner/StormRelDataTypeSystem.java | 37 +
.../apache/storm/sql/planner/StormRelUtils.java | 68 ++
.../planner/UnsupportedOperatorsVisitor.java | 24 +
.../storm/sql/planner/rel/StormCalcRelBase.java | 31 +
.../sql/planner/rel/StormFilterRelBase.java | 31 +
.../storm/sql/planner/rel/StormJoinRelBase.java | 35 +
.../sql/planner/rel/StormProjectRelBase.java | 34 +
.../storm/sql/planner/rel/StormRelNode.java | 24 +
.../planner/rel/StormStreamInsertRelBase.java | 35 +
.../sql/planner/rel/StormStreamScanRelBase.java | 33 +
.../storm/sql/planner/trident/QueryPlanner.java | 156 ++++
.../sql/planner/trident/TridentPlanCreator.java | 123 +++
.../planner/trident/TridentStormRuleSets.java | 110 +++
.../sql/planner/trident/rel/TridentCalcRel.java | 97 +++
.../planner/trident/rel/TridentFilterRel.java | 65 ++
.../trident/rel/TridentLogicalConvention.java | 67 ++
.../planner/trident/rel/TridentProjectRel.java | 69 ++
.../sql/planner/trident/rel/TridentRel.java | 25 +
.../trident/rel/TridentStreamInsertRel.java | 76 ++
.../trident/rel/TridentStreamScanRel.java | 55 ++
.../trident/rules/TridentAggregateRule.java | 39 +
.../planner/trident/rules/TridentCalcRule.java | 45 +
.../trident/rules/TridentFilterRule.java | 45 +
.../planner/trident/rules/TridentJoinRule.java | 37 +
.../trident/rules/TridentModifyRule.java | 71 ++
.../trident/rules/TridentProjectRule.java | 45 +
.../planner/trident/rules/TridentScanRule.java | 58 ++
.../test/org/apache/storm/sql/TestStormSql.java | 492 +++++++++++
.../storm/sql/compiler/TestExprSemantic.java | 410 +++++++++
.../backends/standalone/TestCompilerUtils.java | 183 ++++
.../backends/standalone/TestPlanCompiler.java | 104 +++
.../standalone/TestRelNodeCompiler.java | 64 ++
.../backends/trident/TestCompilerUtils.java | 208 +++++
.../backends/trident/TestPlanCompiler.java | 232 +++++
.../apache/storm/sql/parser/TestSqlParser.java | 54 ++
sql/storm-sql-external/storm-sql-hdfs/pom.xml | 104 +++
.../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 +++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/hdfs/TestHdfsDataSourcesProvider.java | 129 +++
sql/storm-sql-external/storm-sql-kafka/pom.xml | 93 ++
.../sql/kafka/KafkaDataSourcesProvider.java | 170 ++++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/kafka/TestKafkaDataSourcesProvider.java | 125 +++
.../storm-sql-mongodb/pom.xml | 84 ++
.../sql/mongodb/MongoDataSourcesProvider.java | 126 +++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../mongodb/TestMongoDataSourcesProvider.java | 122 +++
sql/storm-sql-external/storm-sql-redis/pom.xml | 76 ++
.../sql/redis/RedisDataSourcesProvider.java | 228 +++++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../sql/redis/TestRedisDataSourcesProvider.java | 158 ++++
sql/storm-sql-runtime/pom.xml | 136 +++
.../calcite/interpreter/StormContext.java | 31 +
.../sql/runtime/AbstractChannelHandler.java | 52 ++
.../sql/runtime/AbstractValuesProcessor.java | 44 +
.../storm/sql/runtime/ChannelContext.java | 30 +
.../storm/sql/runtime/ChannelHandler.java | 41 +
.../org/apache/storm/sql/runtime/Channels.java | 109 +++
.../apache/storm/sql/runtime/DataSource.java | 27 +
.../storm/sql/runtime/DataSourcesProvider.java | 47 +
.../storm/sql/runtime/DataSourcesRegistry.java | 76 ++
.../org/apache/storm/sql/runtime/FieldInfo.java | 47 +
.../storm/sql/runtime/IOutputSerializer.java | 31 +
.../sql/runtime/ISqlTridentDataSource.java | 65 ++
.../sql/runtime/SimpleSqlTridentConsumer.java | 41 +
.../storm/sql/runtime/StormSqlFunctions.java | 34 +
.../calcite/DebuggableExecutableExpression.java | 45 +
.../runtime/calcite/ExecutableExpression.java | 31 +
.../sql/runtime/calcite/StormDataContext.java | 79 ++
.../socket/SocketDataSourcesProvider.java | 100 +++
.../datasource/socket/trident/SocketState.java | 89 ++
.../socket/trident/SocketStateUpdater.java | 59 ++
.../socket/trident/TridentSocketSpout.java | 177 ++++
.../sql/runtime/serde/avro/AvroScheme.java | 74 ++
.../sql/runtime/serde/avro/AvroSerializer.java | 72 ++
.../sql/runtime/serde/avro/CachedSchemas.java | 41 +
.../storm/sql/runtime/serde/csv/CsvScheme.java | 70 ++
.../sql/runtime/serde/csv/CsvSerializer.java | 59 ++
.../sql/runtime/serde/json/JsonScheme.java | 58 ++
.../sql/runtime/serde/json/JsonSerializer.java | 57 ++
.../storm/sql/runtime/serde/tsv/TsvScheme.java | 58 ++
.../sql/runtime/serde/tsv/TsvSerializer.java | 54 ++
.../trident/functions/EvaluationCalc.java | 86 ++
.../trident/functions/EvaluationFilter.java | 61 ++
.../trident/functions/EvaluationFunction.java | 67 ++
.../trident/functions/ForwardFunction.java | 30 +
.../storm/sql/runtime/utils/FieldInfoUtils.java | 39 +
.../storm/sql/runtime/utils/SerdeUtils.java | 123 +++
.../apache/storm/sql/runtime/utils/Utils.java | 55 ++
...apache.storm.sql.runtime.DataSourcesProvider | 16 +
.../apache/storm/sql/TestAvroSerializer.java | 72 ++
.../org/apache/storm/sql/TestCsvSerializer.java | 54 ++
.../storm/sql/TestJsonRepresentation.java | 52 ++
.../org/apache/storm/sql/TestTsvSerializer.java | 46 +
.../test/org/apache/storm/sql/TestUtils.java | 584 +++++++++++++
.../socket/TestSocketDataSourceProvider.java | 94 ++
.../final-package/src/main/assembly/binary.xml | 86 +-
storm-submit-tools/pom.xml | 216 +++++
.../submit/command/DependencyResolverMain.java | 158 ++++
.../storm/submit/dependency/AetherUtils.java | 91 ++
.../apache/storm/submit/dependency/Booter.java | 51 ++
.../submit/dependency/DependencyResolver.java | 98 +++
.../dependency/RepositorySystemFactory.java | 67 ++
.../submit/dependency/AetherUtilsTest.java | 102 +++
.../dependency/DependencyResolverTest.java | 83 ++
414 files changed, 19540 insertions(+), 19605 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/bin/storm-kafka-monitor
----------------------------------------------------------------------
diff --git a/bin/storm-kafka-monitor b/bin/storm-kafka-monitor
index a51052d..2958e85 100755
--- a/bin/storm-kafka-monitor
+++ b/bin/storm-kafka-monitor
@@ -49,4 +49,4 @@ if [ -z "$JAVA_HOME" ]; then
else
JAVA="$JAVA_HOME/bin/java"
fi
-exec $JAVA $STORM_JAAS_CONF_PARAM -cp $STORM_BASE_DIR/toollib/storm-kafka-monitor-*.jar org.apache.storm.kafka.monitor.KafkaOffsetLagUtil "$@"
\ No newline at end of file
+exec $JAVA $STORM_JAAS_CONF_PARAM -cp $STORM_BASE_DIR/lib-tools/storm-kafka-monitor/storm-kafka-monitor-*.jar org.apache.storm.kafka.monitor.KafkaOffsetLagUtil "$@"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index 7ab1f9c..a2d9243 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -82,6 +82,7 @@ if (not os.path.isfile(os.path.join(USER_CONF_DIR, "storm.yaml"))):
STORM_LIB_DIR = os.path.join(STORM_DIR, "lib")
STORM_DRPC_LIB_DIR = os.path.join(STORM_DIR, "lib-drpc-server")
+STORM_TOOLS_LIB_DIR = os.path.join(STORM_DIR, "lib-tools")
STORM_BIN_DIR = os.path.join(STORM_DIR, "bin")
STORM_LOG4J2_CONF_DIR = os.path.join(STORM_DIR, "log4j2")
STORM_SUPERVISOR_LOG_FILE = os.getenv('STORM_SUPERVISOR_LOG_FILE', "supervisor.log")
@@ -167,10 +168,8 @@ def resolve_dependencies(artifacts, artifact_repositories):
print("Resolving dependencies on demand: artifacts (%s) with repositories (%s)" % (artifacts, artifact_repositories))
sys.stdout.flush()
- # TODO: should we move some external modules to outer place?
-
# storm-submit module doesn't rely on storm-core and relevant libs
- extrajars = get_jars_full(STORM_DIR + "/external/storm-submit-tools")
+ extrajars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "submit-tools"))
classpath = normclasspath(os.pathsep.join(extrajars))
command = [
@@ -343,8 +342,8 @@ def sql(sql_file, topology_name):
local_jars = DEP_JARS_OPTS
artifact_to_file_jars = resolve_dependencies(DEP_ARTIFACTS_OPTS, DEP_ARTIFACTS_REPOSITORIES_OPTS)
- sql_core_jars = get_jars_full(STORM_DIR + "/external/sql/storm-sql-core")
- sql_runtime_jars = get_jars_full(STORM_DIR + "/external/sql/storm-sql-runtime")
+ sql_core_jars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "sql", "core"))
+ sql_runtime_jars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "sql", "runtime"))
# include storm-sql-runtime jar(s) to local jar list
local_jars.extend(sql_runtime_jars)
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/README.md
----------------------------------------------------------------------
diff --git a/external/flux/README.md b/external/flux/README.md
deleted file mode 100644
index 206ae22..0000000
--- a/external/flux/README.md
+++ /dev/null
@@ -1,872 +0,0 @@
-# flux
-A framework for creating and deploying Apache Storm streaming computations with less friction.
-
-## Definition
-**flux** |fl\u0259ks| _noun_
-
-1. The action or process of flowing or flowing out
-2. Continuous change
-3. In physics, the rate of flow of a fluid, radiant energy, or particles across a given area
-4. A substance mixed with a solid to lower its melting point
-
-## Rationale
-Bad things happen when configuration is hard-coded. No one should have to recompile or repackage an application in
-order to change configuration.
-
-## About
-Flux is a framework and set of utilities that make defining and deploying Apache Storm topologies less painful and
-deveoper-intensive.
-
-Have you ever found yourself repeating this pattern?:
-
-```java
-
-public static void main(String[] args) throws Exception {
- // logic to determine if we're running locally or not...
- // create necessary config options...
- boolean runLocal = shouldRunLocal();
- if(runLocal){
- LocalCluster cluster = new LocalCluster();
- cluster.submitTopology(name, conf, topology);
- } else {
- StormSubmitter.submitTopology(name, conf, topology);
- }
-}
-```
-
-Wouldn't something like this be easier:
-
-```bash
-storm jar mytopology.jar org.apache.storm.flux.Flux --local config.yaml
-```
-
-or:
-
-```bash
-storm jar mytopology.jar org.apache.storm.flux.Flux --remote config.yaml
-```
-
-Another pain point often mentioned is the fact that the wiring for a Topology graph is often tied up in Java code,
-and that any changes require recompilation and repackaging of the topology jar file. Flux aims to alleviate that
-pain by allowing you to package all your Storm components in a single jar, and use an external text file to define
-the layout and configuration of your topologies.
-
-## Features
-
- * Easily configure and deploy Storm topologies (Both Storm core and Microbatch API) without embedding configuration
- in your topology code
- * Support for existing topology code (see below)
- * Define Storm Core API (Spouts/Bolts) using a flexible YAML DSL
- * YAML DSL support for most Storm components (storm-kafka, storm-hdfs, storm-hbase, etc.)
- * Convenient support for multi-lang components
- * External property substitution/filtering for easily switching between configurations/environments (similar to Maven-style
- `${variable.name}` substitution)
-
-## Usage
-
-To use Flux, add it as a dependency and package all your Storm components in a fat jar, then create a YAML document
-to define your topology (see below for YAML configuration options).
-
-### Building from Source
-The easiest way to use Flux, is to add it as a Maven dependency in you project as described below.
-
-If you would like to build Flux from source and run the unit/integration tests, you will need the following installed
-on your system:
-
-* Python 2.6.x or later
-* Node.js 0.10.x or later
-
-#### Building with unit tests enabled:
-
-```
-mvn clean install
-```
-
-#### Building with unit tests disabled:
-If you would like to build Flux without installing Python or Node.js you can simply skip the unit tests:
-
-```
-mvn clean install -DskipTests=true
-```
-
-Note that if you plan on using Flux to deploy topologies to a remote cluster, you will still need to have Python
-installed since it is required by Apache Storm.
-
-
-#### Building with integration tests enabled:
-
-```
-mvn clean install -DskipIntegration=false
-```
-
-
-### Packaging with Maven
-To enable Flux for your Storm components, you need to add it as a dependency such that it's included in the Storm
-topology jar. This can be accomplished with the Maven shade plugin (preferred) or the Maven assembly plugin (not
-recommended).
-
-#### Flux Maven Dependency
-The current version of Flux is available in Maven Central at the following coordinates:
-```xml
-<dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-core</artifactId>
- <version>${storm.version}</version>
-</dependency>
-```
-
-Using shell spouts and bolts requires additional Flux Wrappers library:
-```xml
-<dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-wrappers</artifactId>
- <version>${storm.version}</version>
-</dependency>
-```
-
-#### Creating a Flux-Enabled Topology JAR
-The example below illustrates Flux usage with the Maven shade plugin:
-
- ```xml
-<!-- include Flux and user dependencies in the shaded jar -->
-<dependencies>
- <!-- Flux include -->
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-core</artifactId>
- <version>${storm.version}</version>
- </dependency>
- <!-- Flux Wrappers include -->
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-wrappers</artifactId>
- <version>${storm.version}</version>
- </dependency>
-
- <!-- add user dependencies here... -->
-
-</dependencies>
-<!-- create a fat jar that includes all dependencies -->
-<build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <version>1.4</version>
- <configuration>
- <createDependencyReducedPom>true</createDependencyReducedPom>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <transformers>
- <transformer
- implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
- <transformer
- implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- <mainClass>org.apache.storm.flux.Flux</mainClass>
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
-</build>
- ```
-
-### Deploying and Running a Flux Topology
-Once your topology components are packaged with the Flux dependency, you can run different topologies either locally
-or remotely using the `storm jar` command. For example, if your fat jar is named `myTopology-0.1.0-SNAPSHOT.jar` you
-could run it locally with the command:
-
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml
-
-```
-
-### Command line options
-```
-usage: storm jar <my_topology_uber_jar.jar> org.apache.storm.flux.Flux
- [options] <topology-config.yaml>
- -d,--dry-run Do not run or deploy the topology. Just
- build, validate, and print information about
- the topology.
- -e,--env-filter Perform environment variable substitution.
- Replace keys identified with `${ENV-[NAME]}`
- will be replaced with the corresponding
- `NAME` environment value
- -f,--filter <file> Perform property substitution. Use the
- specified file as a source of properties,
- and replace keys identified with {$[property
- name]} with the value defined in the
- properties file.
- -i,--inactive Deploy the topology, but do not activate it.
- -l,--local Run the topology in local mode.
- -n,--no-splash Suppress the printing of the splash screen.
- -q,--no-detail Suppress the printing of topology details.
- -r,--remote Deploy the topology to a remote cluster.
- -R,--resource Treat the supplied path as a classpath
- resource instead of a file.
- -s,--sleep <ms> When running locally, the amount of time to
- sleep (in ms.) before killing the topology
- and shutting down the local cluster.
- -z,--zookeeper <host:port> When running in local mode, use the
- ZooKeeper at the specified <host>:<port>
- instead of the in-process ZooKeeper.
- (requires Storm 0.9.3 or later)
-```
-
-**NOTE:** Flux tries to avoid command line switch collision with the `storm` command, and allows any other command line
-switches to pass through to the `storm` command.
-
-For example, you can use the `storm` command switch `-c` to override a topology configuration property. The following
-example command will run Flux and override the `nimus.host` configuration:
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --remote my_config.yaml -c nimbus.host=localhost
-```
-
-### Sample output
-```
-\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557 \u2588\u2588\u2557\u2588\u2588\u2557 \u2588\u2588\u2557
-\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
-\u2588\u2588\u2588\u2588\u2588\u2557 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
-\u2588\u2588\u2554\u2550\u2550\u255d \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
-\u2588\u2588\u2551 \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
-\u255a\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d \u255a\u2550\u255d
-+- Apache Storm -+
-+- data FLow User eXperience -+
-Version: 0.3.0
-Parsing file: /Users/hsimpson/Projects/donut_domination/storm/shell_test.yaml
----------- TOPOLOGY DETAILS ----------
-Name: shell-topology
---------------- SPOUTS ---------------
-sentence-spout[1](org.apache.storm.flux.wrappers.spouts.FluxShellSpout)
----------------- BOLTS ---------------
-splitsentence[1](org.apache.storm.flux.wrappers.bolts.FluxShellBolt)
-log[1](org.apache.storm.flux.wrappers.bolts.LogInfoBolt)
-count[1](org.apache.storm.testing.TestWordCounter)
---------------- STREAMS ---------------
-sentence-spout --SHUFFLE--> splitsentence
-splitsentence --FIELDS--> count
-count --SHUFFLE--> log
---------------------------------------
-Submitting topology: 'shell-topology' to remote cluster...
-```
-
-## YAML Configuration
-Flux topologies are defined in a YAML file that describes a topology. A Flux topology
-definition consists of the following:
-
- 1. A topology name
- 2. A list of topology "components" (named Java objects that will be made available in the environment)
- 3. **EITHER** (A DSL topology definition):
- * A list of spouts, each identified by a unique ID
- * A list of bolts, each identified by a unique ID
- * A list of "stream" objects representing a flow of tuples between spouts and bolts
- 4. **OR** (A JVM class that can produce a `org.apache.storm.generated.StormTopology` instance:
- * A `topologySource` definition.
-
-
-
-For example, here is a simple definition of a wordcount topology using the YAML DSL:
-
-```yaml
-name: "yaml-topology"
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- - id: "bolt-2"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
-#stream definitions
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "bolt-1 --> bolt2"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: SHUFFLE
-
-
-```
-## Property Substitution/Filtering
-It's common for developers to want to easily switch between configurations, for example switching deployment between
-a development environment and a production environment. This can be accomplished by using separate YAML configuration
-files, but that approach would lead to unnecessary duplication, especially in situations where the Storm topology
-does not change, but configuration settings such as host names, ports, and parallelism paramters do.
-
-For this case, Flux offers properties filtering to allow you two externalize values to a `.properties` file and have
-them substituted before the `.yaml` file is parsed.
-
-To enable property filtering, use the `--filter` command line option and specify a `.properties` file. For example,
-if you invoked flux like so:
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml --filter dev.properties
-```
-With the following `dev.properties` file:
-
-```properties
-kafka.zookeeper.hosts: localhost:2181
-```
-
-You would then be able to reference those properties by key in your `.yaml` file using `${}` syntax:
-
-```yaml
- - id: "zkHosts"
- className: "org.apache.storm.kafka.ZkHosts"
- constructorArgs:
- - "${kafka.zookeeper.hosts}"
-```
-
-In this case, Flux would replace `${kafka.zookeeper.hosts}` with `localhost:2181` before parsing the YAML contents.
-
-### Environment Variable Substitution/Filtering
-Flux also allows environment variable substitution. For example, if an environment variable named `ZK_HOSTS` if defined,
-you can reference it in a Flux YAML file with the following syntax:
-
-```
-${ENV-ZK_HOSTS}
-```
-
-## Components
-Components are essentially named object instances that are made available as configuration options for spouts and
-bolts. If you are familiar with the Spring framework, components are roughly analagous to Spring beans.
-
-Every component is identified, at a minimum, by a unique identifier (String) and a class name (String). For example,
-the following will make an instance of the `org.apache.storm.kafka.StringScheme` class available as a reference under the key
-`"stringScheme"` . This assumes the `org.apache.storm.kafka.StringScheme` has a default constructor.
-
-```yaml
-components:
- - id: "stringScheme"
- className: "org.apache.storm.kafka.StringScheme"
-```
-
-### Contructor Arguments, References, Properties and Configuration Methods
-
-####Constructor Arguments
-Arguments to a class constructor can be configured by adding a `contructorArgs` element to a components.
-`constructorArgs` is a list of objects that will be passed to the class' constructor. The following example creates an
-object by calling the constructor that takes a single string as an argument:
-
-```yaml
- - id: "zkHosts"
- className: "org.apache.storm.kafka.ZkHosts"
- constructorArgs:
- - "localhost:2181"
- - true
-```
-
-####References
-Each component instance is identified by a unique id that allows it to be used/reused by other components. To
-reference an existing component, you specify the id of the component with the `ref` tag.
-
-In the following example, a component with the id `"stringScheme"` is created, and later referenced, as a an argument
-to another component's constructor:
-
-```yaml
-components:
- - id: "stringScheme"
- className: "org.apache.storm.kafka.StringScheme"
-
- - id: "stringMultiScheme"
- className: "org.apache.storm.spout.SchemeAsMultiScheme"
- constructorArgs:
- - ref: "stringScheme" # component with id "stringScheme" must be declared above.
-```
-
-You can also reference existing components in list via specifying the id of the components with the `reflist` tag.
-The type of the reflist will be `List<Object>`, but Flux can automatically convert List to Array (also varargs),
-so you can use reflist on argument which type is `List<Type>`, or `Type[]`, or `Type...`.
-
-Please note that all components in the list must be same type.
-
-```yaml
-components:
- - id: "boundCQLStatementMapperBuilder"
- className: "org.apache.storm.cassandra.query.builder.BoundCQLStatementMapperBuilder"
- constructorArgs:
- - "INSERT INTO sink_cassandra (eventKey, driverId, truckId, driverName) VALUES (?, ?, ?, ?)"
- configMethods:
- - name: "bind"
- args:
- - reflist: ["FieldSelector-1", "FieldSelector-2", "FieldSelector-3", "FieldSelector-4"]
-```
-
-**N.B.:** References can only be used after (below) the object they point to has been declared.
-
-####Properties
-In addition to calling constructors with different arguments, Flux also allows you to configure components using
-JavaBean-like setter methods and fields declared as `public`:
-
-```yaml
- - id: "spoutConfig"
- className: "org.apache.storm.kafka.SpoutConfig"
- constructorArgs:
- # brokerHosts
- - ref: "zkHosts"
- # topic
- - "myKafkaTopic"
- # zkRoot
- - "/kafkaSpout"
- # id
- - "myId"
- properties:
- - name: "ignoreZkOffsets"
- value: true
- - name: "scheme"
- ref: "stringMultiScheme"
-```
-
-In the example above, the `properties` declaration will cause Flux to look for a public method in the `SpoutConfig` with
-the signature `setForceFromStart(boolean b)` and attempt to invoke it. If a setter method is not found, Flux will then
-look for a public instance variable with the name `ignoreZkOffsets` and attempt to set its value.
-
-References may also be used as property values.
-
-####Configuration Methods
-Conceptually, configuration methods are similar to Properties and Constructor Args -- they allow you to invoke an
-arbitrary method on an object after it is constructed. Configuration methods are useful for working with classes that
-don't expose JavaBean methods or have constructors that can fully configure the object. Common examples include classes
-that use the builder pattern for configuration/composition.
-
-The following YAML example creates a bolt and configures it by calling several methods:
-
-```yaml
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.flux.test.TestBolt"
- parallelism: 1
- configMethods:
- - name: "withFoo"
- args:
- - "foo"
- - name: "withNone"
- # no args needed, so no "args" line
- - name: "withBar"
- args:
- - "bar"
- - name: "withFooBar"
- args:
- - "foo"
- - "bar"
-```
-
-The signatures of the corresponding methods are as follows:
-
-```java
- public void withFoo(String foo);
- public void withNone(); // method with zero arguments
- public void withBar(String bar);
- public void withFooBar(String foo, String bar);
-```
-
-Arguments passed to configuration methods work much the same way as constructor arguments, and support references as
-well.
-
-### Using Java `enum`s in Contructor Arguments, References, Properties and Configuration Methods
-You can easily use Java `enum` values as arguments in a Flux YAML file, simply by referencing the name of the `enum`.
-
-For example, [Storm's HDFS module]() includes the following `enum` definition (simplified for brevity):
-
-```java
-public static enum Units {
- KB, MB, GB, TB
-}
-```
-
-And the `org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy` class has the following constructor:
-
-```java
-public FileSizeRotationPolicy(float count, Units units)
-
-```
-The following Flux `component` definition could be used to call the constructor:
-
-```yaml
- - id: "rotationPolicy"
- className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
- constructorArgs:
- - 5.0
- - MB
-```
-
-The above definition is functionally equivalent to the following Java code:
-
-```java
-// rotate files when they reach 5MB
-FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(5.0f, Units.MB);
-```
-
-## Topology Config
-The `config` section is simply a map of Storm topology configuration parameters that will be passed to the
-`org.apache.storm.StormSubmitter` as an instance of the `org.apache.storm.Config` class:
-
-```yaml
-config:
- topology.workers: 4
- topology.max.spout.pending: 1000
- topology.message.timeout.secs: 30
-```
-
-# Existing Topologies
-If you have existing Storm topologies, you can still use Flux to deploy/run/test them. This feature allows you to
-leverage Flux Constructor Arguments, References, Properties, and Topology Config declarations for existing topology
-classes.
-
-The easiest way to use an existing topology class is to define
-a `getTopology()` instance method with one of the following signatures:
-
-```java
-public StormTopology getTopology(Map<String, Object> config)
-```
-or:
-
-```java
-public StormTopology getTopology(Config config)
-```
-
-You could then use the following YAML to configure your topology:
-
-```yaml
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopology"
-```
-
-If the class you would like to use as a topology source has a different method name (i.e. not `getTopology`), you can
-override it:
-
-```yaml
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopology"
- methodName: "getTopologyWithDifferentMethodName"
-```
-
-__N.B.:__ The specified method must accept a single argument of type `java.util.Map<String, Object>` or
-`org.apache.storm.Config`, and return a `org.apache.storm.generated.StormTopology` object.
-
-# YAML DSL
-## Spouts and Bolts
-Spout and Bolts are configured in their own respective section of the YAML configuration. Spout and Bolt definitions
-are extensions to the `component` definition that add a `parallelism` parameter that sets the parallelism for a
-component when the topology is deployed.
-
-Because spout and bolt definitions extend `component` they support constructor arguments, references, and properties as
-well.
-
-Shell spout example:
-
-```yaml
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- parallelism: 1
-```
-
-Kafka spout example:
-
-```yaml
-components:
- - id: "stringScheme"
- className: "org.apache.storm.kafka.StringScheme"
-
- - id: "stringMultiScheme"
- className: "org.apache.storm.spout.SchemeAsMultiScheme"
- constructorArgs:
- - ref: "stringScheme"
-
- - id: "zkHosts"
- className: "org.apache.storm.kafka.ZkHosts"
- constructorArgs:
- - "localhost:2181"
-
-# Alternative kafka config
-# - id: "kafkaConfig"
-# className: "org.apache.storm.kafka.KafkaConfig"
-# constructorArgs:
-# # brokerHosts
-# - ref: "zkHosts"
-# # topic
-# - "myKafkaTopic"
-# # clientId (optional)
-# - "myKafkaClientId"
-
- - id: "spoutConfig"
- className: "org.apache.storm.kafka.SpoutConfig"
- constructorArgs:
- # brokerHosts
- - ref: "zkHosts"
- # topic
- - "myKafkaTopic"
- # zkRoot
- - "/kafkaSpout"
- # id
- - "myId"
- properties:
- - name: "ignoreZkOffsets"
- value: true
- - name: "scheme"
- ref: "stringMultiScheme"
-
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "kafka-spout"
- className: "org.apache.storm.kafka.KafkaSpout"
- constructorArgs:
- - ref: "spoutConfig"
-
-```
-
-Bolt Examples:
-
-```yaml
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-```
-## Streams and Stream Groupings
-Streams in Flux are represented as a list of connections (Graph edges, data flow, etc.) between the Spouts and Bolts in
-a topology, with an associated Grouping definition.
-
-A Stream definition has the following properties:
-
-**`name`:** A name for the connection (optional, currently unused)
-
-**`from`:** The `id` of a Spout or Bolt that is the source (publisher)
-
-**`to`:** The `id` of a Spout or Bolt that is the destination (subscriber)
-
-**`grouping`:** The stream grouping definition for the Stream
-
-A Grouping definition has the following properties:
-
-**`type`:** The type of grouping. One of `ALL`,`CUSTOM`,`DIRECT`,`SHUFFLE`,`LOCAL_OR_SHUFFLE`,`FIELDS`,`GLOBAL`, or `NONE`.
-
-**`streamId`:** The Storm stream ID (Optional. If unspecified will use the default stream)
-
-**`args`:** For the `FIELDS` grouping, a list of field names.
-
-**`customClass`** For the `CUSTOM` grouping, a definition of custom grouping class instance
-
-The `streams` definition example below sets up a topology with the following wiring:
-
-```
- kafka-spout --> splitsentence --> count --> log
-```
-
-
-```yaml
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "kafka-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
-```
-
-### Custom Stream Groupings
-Custom stream groupings are defined by setting the grouping type to `CUSTOM` and defining a `customClass` parameter
-that tells Flux how to instantiate the custom class. The `customClass` definition extends `component`, so it supports
-constructor arguments, references, and properties as well.
-
-The example below creates a Stream with an instance of the `org.apache.storm.testing.NGrouping` custom stream grouping
-class.
-
-```yaml
- - name: "bolt-1 --> bolt2"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: CUSTOM
- customClass:
- className: "org.apache.storm.testing.NGrouping"
- constructorArgs:
- - 1
-```
-
-## Includes and Overrides
-Flux allows you to include the contents of other YAML files, and have them treated as though they were defined in the
-same file. Includes may be either files, or classpath resources.
-
-Includes are specified as a list of maps:
-
-```yaml
-includes:
- - resource: false
- file: "src/test/resources/configs/shell_test.yaml"
- override: false
-```
-
-If the `resource` property is set to `true`, the include will be loaded as a classpath resource from the value of the
-`file` attribute, otherwise it will be treated as a regular file.
-
-The `override` property controls how includes affect the values defined in the current file. If `override` is set to
-`true`, values in the included file will replace values in the current file being parsed. If `override` is set to
-`false`, values in the current file being parsed will take precedence, and the parser will refuse to replace them.
-
-**N.B.:** Includes are not yet recursive. Includes from included files will be ignored.
-
-
-## Basic Word Count Example
-
-This example uses a spout implemented in JavaScript, a bolt implemented in Python, and a bolt implemented in Java
-
-Topology YAML config:
-
-```yaml
----
-name: "shell-topology"
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "sentence-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
-```
-
-
-## Micro-Batching (Trident) API Support
-Currenty, the Flux YAML DSL only supports the Core Storm API, but support for Storm's micro-batching API is planned.
-
-To use Flux with a Trident topology, define a topology getter method and reference it in your YAML config:
-
-```yaml
-name: "my-trident-topology"
-
-config:
- topology.workers: 1
-
-topologySource:
- className: "org.apache.storm.flux.test.TridentTopologySource"
- # Flux will look for "getTopology", this will override that.
- methodName: "getTopologyWithDifferentMethodName"
-```
-
-## Committer Sponsors
-
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/pom.xml b/external/flux/flux-core/pom.xml
deleted file mode 100644
index 1cf4a9b..0000000
--- a/external/flux/flux-core/pom.xml
+++ /dev/null
@@ -1,112 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux</artifactId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../pom.xml</relativePath>
- </parent>
-
- <artifactId>flux-core</artifactId>
- <packaging>jar</packaging>
-
- <name>flux-core</name>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>flux-wrappers</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-kafka</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-hdfs</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-hbase</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.yaml</groupId>
- <artifactId>snakeyaml</artifactId>
- </dependency>
- </dependencies>
- <build>
- <resources>
- <resource>
- <directory>src/main/resources</directory>
- <filtering>true</filtering>
- </resource>
- </resources>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <configuration>
- <createDependencyReducedPom>true</createDependencyReducedPom>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.sf</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.dsa</exclude>
- <exclude>META-INF/*.RSA</exclude>
- <exclude>META-INF/*.rsa</exclude>
- <exclude>META-INF/*.EC</exclude>
- <exclude>META-INF/*.ec</exclude>
- <exclude>META-INF/MSFTSIG.SF</exclude>
- <exclude>META-INF/MSFTSIG.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
- <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- <mainClass>org.apache.storm.flux.Flux</mainClass>
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
deleted file mode 100644
index 982765f..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux;
-
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.generated.TopologyInitialStatus;
-import org.apache.storm.utils.Utils;
-import org.apache.commons.cli.*;
-import org.apache.storm.flux.model.*;
-import org.apache.storm.flux.parser.FluxParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Flux entry point.
- *
- */
-public class Flux {
- private static final Logger LOG = LoggerFactory.getLogger(Flux.class);
-
- private static final Long DEFAULT_LOCAL_SLEEP_TIME = 60000l;
-
- private static final Long DEFAULT_ZK_PORT = 2181l;
-
- private static final String OPTION_LOCAL = "local";
- private static final String OPTION_REMOTE = "remote";
- private static final String OPTION_RESOURCE = "resource";
- private static final String OPTION_SLEEP = "sleep";
- private static final String OPTION_DRY_RUN = "dry-run";
- private static final String OPTION_NO_DETAIL = "no-detail";
- private static final String OPTION_NO_SPLASH = "no-splash";
- private static final String OPTION_INACTIVE = "inactive";
- private static final String OPTION_ZOOKEEPER = "zookeeper";
- private static final String OPTION_FILTER = "filter";
- private static final String OPTION_ENV_FILTER = "env-filter";
-
- public static void main(String[] args) throws Exception {
- Options options = new Options();
-
- options.addOption(option(0, "l", OPTION_LOCAL, "Run the topology in local mode."));
-
- options.addOption(option(0, "r", OPTION_REMOTE, "Deploy the topology to a remote cluster."));
-
- options.addOption(option(0, "R", OPTION_RESOURCE, "Treat the supplied path as a classpath resource instead of a file."));
-
- options.addOption(option(1, "s", OPTION_SLEEP, "ms", "When running locally, the amount of time to sleep (in ms.) " +
- "before killing the topology and shutting down the local cluster."));
-
- options.addOption(option(0, "d", OPTION_DRY_RUN, "Do not run or deploy the topology. Just build, validate, " +
- "and print information about the topology."));
-
- options.addOption(option(0, "q", OPTION_NO_DETAIL, "Suppress the printing of topology details."));
-
- options.addOption(option(0, "n", OPTION_NO_SPLASH, "Suppress the printing of the splash screen."));
-
- options.addOption(option(0, "i", OPTION_INACTIVE, "Deploy the topology, but do not activate it."));
-
- options.addOption(option(1, "z", OPTION_ZOOKEEPER, "host:port", "When running in local mode, use the ZooKeeper at the " +
- "specified <host>:<port> instead of the in-process ZooKeeper. (requires Storm 0.9.3 or later)"));
-
- options.addOption(option(1, "f", OPTION_FILTER, "file", "Perform property substitution. Use the specified file " +
- "as a source of properties, and replace keys identified with {$[property name]} with the value defined " +
- "in the properties file."));
-
- options.addOption(option(0, "e", OPTION_ENV_FILTER, "Perform environment variable substitution. Replace keys" +
- "identified with `${ENV-[NAME]}` will be replaced with the corresponding `NAME` environment value"));
-
- CommandLineParser parser = new BasicParser();
- CommandLine cmd = parser.parse(options, args);
-
- if (cmd.getArgs().length != 1) {
- usage(options);
- System.exit(1);
- }
- runCli(cmd);
- }
-
- private static Option option(int argCount, String shortName, String longName, String description){
- return option(argCount, shortName, longName, longName, description);
- }
-
- private static Option option(int argCount, String shortName, String longName, String argName, String description){
- Option option = OptionBuilder.hasArgs(argCount)
- .withArgName(argName)
- .withLongOpt(longName)
- .withDescription(description)
- .create(shortName);
- return option;
- }
-
- private static void usage(Options options) {
- HelpFormatter formatter = new HelpFormatter();
- formatter.printHelp("storm jar <my_topology_uber_jar.jar> " +
- Flux.class.getName() +
- " [options] <topology-config.yaml>", options);
- }
-
- private static void runCli(CommandLine cmd)throws Exception {
- if(!cmd.hasOption(OPTION_NO_SPLASH)) {
- printSplash();
- }
-
- boolean dumpYaml = cmd.hasOption("dump-yaml");
-
- TopologyDef topologyDef = null;
- String filePath = (String)cmd.getArgList().get(0);
-
- // TODO conditionally load properties from a file our resource
- String filterProps = null;
- if(cmd.hasOption(OPTION_FILTER)){
- filterProps = cmd.getOptionValue(OPTION_FILTER);
- }
-
-
- boolean envFilter = cmd.hasOption(OPTION_ENV_FILTER);
- if(cmd.hasOption(OPTION_RESOURCE)){
- printf("Parsing classpath resource: %s", filePath);
- topologyDef = FluxParser.parseResource(filePath, dumpYaml, true, filterProps, envFilter);
- } else {
- printf("Parsing file: %s",
- new File(filePath).getAbsolutePath());
- topologyDef = FluxParser.parseFile(filePath, dumpYaml, true, filterProps, envFilter);
- }
-
-
- String topologyName = topologyDef.getName();
- // merge contents of `config` into topology config
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
-
- if(!cmd.hasOption(OPTION_NO_DETAIL)){
- printTopologyInfo(context);
- }
-
- if(!cmd.hasOption(OPTION_DRY_RUN)) {
- if (cmd.hasOption(OPTION_REMOTE)) {
- LOG.info("Running remotely...");
- // should the topology be active or inactive
- SubmitOptions submitOptions = null;
- if(cmd.hasOption(OPTION_INACTIVE)){
- LOG.info("Deploying topology in an INACTIVE state...");
- submitOptions = new SubmitOptions(TopologyInitialStatus.INACTIVE);
- } else {
- LOG.info("Deploying topology in an ACTIVE state...");
- submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
- }
- StormSubmitter.submitTopology(topologyName, conf, topology, submitOptions, null);
- } else {
- LOG.info("Running in local mode...");
-
- String sleepStr = cmd.getOptionValue(OPTION_SLEEP);
- Long sleepTime = DEFAULT_LOCAL_SLEEP_TIME;
- if (sleepStr != null) {
- sleepTime = Long.parseLong(sleepStr);
- }
- LOG.debug("Sleep time: {}", sleepTime);
- LocalCluster cluster = null;
-
- // in-process or external zookeeper
- if(cmd.hasOption(OPTION_ZOOKEEPER)){
- String zkStr = cmd.getOptionValue(OPTION_ZOOKEEPER);
- LOG.info("Using ZooKeeper at '{}' instead of in-process one.", zkStr);
- long zkPort = DEFAULT_ZK_PORT;
- String zkHost = null;
- if(zkStr.contains(":")){
- String[] hostPort = zkStr.split(":");
- zkHost = hostPort[0];
- zkPort = hostPort.length > 1 ? Long.parseLong(hostPort[1]) : DEFAULT_ZK_PORT;
-
- } else {
- zkHost = zkStr;
- }
- // the following constructor is only available in 0.9.3 and later
- try {
- cluster = new LocalCluster(zkHost, zkPort);
- } catch (NoSuchMethodError e){
- LOG.error("The --zookeeper option can only be used with Apache Storm 0.9.3 and later.");
- System.exit(1);
- }
- } else {
- cluster = new LocalCluster();
- }
- try (LocalTopology topo = cluster.submitTopology(topologyName, conf, topology)) {
- Utils.sleep(sleepTime);
- } finally {
- cluster.shutdown();
- }
- }
- }
- }
-
- static void printTopologyInfo(ExecutionContext ctx){
- TopologyDef t = ctx.getTopologyDef();
- if(t.isDslTopology()) {
- print("---------- TOPOLOGY DETAILS ----------");
-
- printf("Topology Name: %s", t.getName());
- print("--------------- SPOUTS ---------------");
- for (SpoutDef s : t.getSpouts()) {
- printf("%s [%d] (%s)", s.getId(), s.getParallelism(), s.getClassName());
- }
- print("---------------- BOLTS ---------------");
- for (BoltDef b : t.getBolts()) {
- printf("%s [%d] (%s)", b.getId(), b.getParallelism(), b.getClassName());
- }
-
- print("--------------- STREAMS ---------------");
- for (StreamDef sd : t.getStreams()) {
- printf("%s --%s--> %s", sd.getFrom(), sd.getGrouping().getType(), sd.getTo());
- }
- print("--------------------------------------");
- }
- }
-
- // save a little typing
- private static void printf(String format, Object... args){
- print(String.format(format, args));
- }
-
- private static void print(String string){
- System.out.println(string);
- }
-
- private static void printSplash() throws IOException {
- // banner
- InputStream is = Flux.class.getResourceAsStream("/splash.txt");
- if(is != null){
- InputStreamReader isr = new InputStreamReader(is, "UTF-8");
- BufferedReader br = new BufferedReader(isr);
- String line = null;
- while((line = br.readLine()) != null){
- System.out.println(line);
- }
- }
- }
-}
[23/23] storm git commit: STORM-2453: CHANGELOG
Posted by ka...@apache.org.
STORM-2453: CHANGELOG
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/c652d3ff
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/c652d3ff
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/c652d3ff
Branch: refs/heads/master
Commit: c652d3ffc2e9a701f0fd67f787f9e257499d6cbb
Parents: 1dd4bbc
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Apr 6 08:18:49 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Apr 6 08:18:49 2017 +0900
----------------------------------------------------------------------
CHANGELOG.md | 1 +
1 file changed, 1 insertion(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/c652d3ff/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 846823d..7ff80c1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
\ufeff## 2.0.0
+ * STORM-2453: Move non-connectors into the top directory
* STORM-2363: Provide configuration to set the number of RollingWindow
* STORM-2058: Fix Maven warnings about missing reporting.plugins.plugin.version
* STORM-1369: Add MapState implementation to storm-cassandra
[19/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
deleted file mode 100644
index bb5d7ec..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
+++ /dev/null
@@ -1,256 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.flux.model.ExecutionContext;
-import org.apache.storm.flux.model.TopologyDef;
-import org.apache.storm.flux.parser.FluxParser;
-import org.apache.storm.flux.test.TestBolt;
-import org.junit.Test;
-
-import java.io.File;
-
-import static org.junit.Assert.*;
-
-public class TCKTest {
- @Test
- public void testTCK() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/tck.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testShellComponents() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/shell_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testBadShellComponents() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_shell_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testKafkaSpoutConfig() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testLoadFromResource() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
-
- @Test
- public void testHdfs() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/hdfs_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testDiamondTopology() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/diamond-topology.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
-
- @Test
- public void testHbase() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/simple_hbase.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testBadHbase() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_hbase.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testIncludes() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/include_test.yaml", false, true, null, false);
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- assertTrue(topologyDef.getName().equals("include-topology"));
- assertTrue(topologyDef.getBolts().size() > 0);
- assertTrue(topologyDef.getSpouts().size() > 0);
- topology.validate();
- }
-
- @Test
- public void testTopologySource() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testTopologySourceWithReflection() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testTopologySourceWithConfigParam() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection-config.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testTopologySourceWithMethodName() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-method-override.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
-
- @Test
- public void testTridentTopologySource() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-trident.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testInvalidTopologySource() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/invalid-existing-topology.yaml", false, true, null, false);
- assertFalse("Topology config is invalid.", topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- }
-
-
- @Test
- public void testTopologySourceWithGetMethodName() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
- }
-
- @Test
- public void testTopologySourceWithConfigMethods() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/config-methods-test.yaml", false, true, null, false);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
-
- // make sure the property was actually set
- TestBolt bolt = (TestBolt)context.getBolt("bolt-1");
- assertTrue(bolt.getFoo().equals("foo"));
- assertTrue(bolt.getBar().equals("bar"));
- assertTrue(bolt.getFooBar().equals("foobar"));
- assertArrayEquals(new TestBolt.TestClass[] {new TestBolt.TestClass("foo"), new TestBolt.TestClass("bar"), new TestBolt.TestClass("baz")}, bolt.getClasses());
- }
-
- @Test
- public void testVariableSubstitution() throws Exception {
- TopologyDef topologyDef = FluxParser.parseResource("/configs/substitution-test.yaml", false, true, "src/test/resources/configs/test.properties", true);
- assertTrue(topologyDef.validate());
- Config conf = FluxBuilder.buildConfig(topologyDef);
- ExecutionContext context = new ExecutionContext(topologyDef, conf);
- StormTopology topology = FluxBuilder.buildTopology(context);
- assertNotNull(topology);
- topology.validate();
-
- // test basic substitution
- assertEquals("Property not replaced.",
- "substitution-topology",
- context.getTopologyDef().getName());
-
- // test environment variable substitution
- // $PATH should be defined on most systems
- String envPath = System.getenv().get("PATH");
- assertEquals("ENV variable not replaced.",
- envPath,
- context.getTopologyDef().getConfig().get("test.env.value"));
-
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
deleted file mode 100644
index dcded17..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.multilang;
-
-
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Sanity checks to make sure we can at least invoke the shells used.
- */
-public class MultilangEnvirontmentTest {
- private static final Logger LOG = LoggerFactory.getLogger(MultilangEnvirontmentTest.class);
-
- @Test
- public void testInvokePython() throws Exception {
- String[] command = new String[]{"python", "--version"};
- int exitVal = invokeCommand(command);
- assertEquals("Exit value for python is 0.", 0, exitVal);
- }
-
- @Test
- public void testInvokeNode() throws Exception {
- String[] command = new String[]{"node", "--version"};
- int exitVal = invokeCommand(command);
- assertEquals("Exit value for node is 0.", 0, exitVal);
- }
-
- private static class StreamRedirect implements Runnable {
- private InputStream in;
- private OutputStream out;
-
- public StreamRedirect(InputStream in, OutputStream out) {
- this.in = in;
- this.out = out;
- }
-
- @Override
- public void run() {
- try {
- int i = -1;
- while ((i = this.in.read()) != -1) {
- out.write(i);
- }
- this.in.close();
- this.out.close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-
- private int invokeCommand(String[] args) throws Exception {
- LOG.debug("Invoking command: {}", args);
-
- ProcessBuilder pb = new ProcessBuilder(args);
- pb.redirectErrorStream(true);
- final Process proc = pb.start();
-
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- Thread t = new Thread(new StreamRedirect(proc.getInputStream(), out));
- t.start();
- int exitVal = proc.waitFor();
- LOG.debug("Command result: {}", out.toString());
- return exitVal;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
deleted file mode 100644
index 8e3cda2..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-/**
- * Test topology source that does not implement TopologySource, but has the same
- * `getTopology()` method.
- */
-public class SimpleTopology{
-
-
- public SimpleTopology(){}
-
- public SimpleTopology(String foo, String bar){}
-
- public StormTopology getTopologyWithDifferentMethodName(Map<String, Object> config){
- return getTopology(config);
- }
-
-
- public StormTopology getTopology(Map<String, Object> config) {
- TopologyBuilder builder = new TopologyBuilder();
-
- // spouts
- FluxShellSpout spout = new FluxShellSpout(
- new String[]{"node", "randomsentence.js"},
- new String[]{"word"});
- builder.setSpout("sentence-spout", spout, 1);
-
- // bolts
- builder.setBolt("log-bolt", new LogInfoBolt(), 1)
- .shuffleGrouping("sentence-spout");
-
- return builder.createTopology();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
deleted file mode 100644
index 2fadacf..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-public class SimpleTopologySource implements TopologySource {
-
-
- public SimpleTopologySource(){}
-
- public SimpleTopologySource(String foo, String bar){}
-
-
- @Override
- public StormTopology getTopology(Map<String, Object> config) {
- TopologyBuilder builder = new TopologyBuilder();
-
- // spouts
- FluxShellSpout spout = new FluxShellSpout(
- new String[]{"node", "randomsentence.js"},
- new String[]{"word"});
- builder.setSpout("sentence-spout", spout, 1);
-
- // bolts
- builder.setBolt("log-bolt", new LogInfoBolt(), 1)
- .shuffleGrouping("sentence-spout");
-
- return builder.createTopology();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
deleted file mode 100644
index 8b0aa05..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-/**
- * Test topology source that does not implement TopologySource, but has the same
- * `getTopology()` method.
- */
-public class SimpleTopologyWithConfigParam {
-
-
- public SimpleTopologyWithConfigParam(){}
-
- public SimpleTopologyWithConfigParam(String foo, String bar){}
-
-
- public StormTopology getTopology(Config config) {
- TopologyBuilder builder = new TopologyBuilder();
-
- // spouts
- FluxShellSpout spout = new FluxShellSpout(
- new String[]{"node", "randomsentence.js"},
- new String[]{"word"});
- builder.setSpout("sentence-spout", spout, 1);
-
- // bolts
- builder.setBolt("log-bolt", new LogInfoBolt(), 1)
- .shuffleGrouping("sentence-spout");
-
- return builder.createTopology();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
deleted file mode 100644
index 28d11b6..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-
-
-public class TestBolt extends BaseBasicBolt {
- private static final Logger LOG = LoggerFactory.getLogger(TestBolt.class);
-
- private String foo;
- private String bar;
- private String fooBar;
- private String none;
- private TestClass[] classes;
-
- public static class TestClass implements Serializable {
- private String field;
-
- public TestClass(String field) {
- this.field = field;
- }
-
- public String getField() {
- return field;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (!(o instanceof TestClass)) return false;
-
- TestClass testClass = (TestClass) o;
-
- return getField() != null ? getField().equals(testClass.getField()) : testClass.getField() == null;
- }
-
- @Override
- public int hashCode() {
- return getField() != null ? getField().hashCode() : 0;
- }
- }
-
-
- public static enum TestEnum {
- FOO,
- BAR
- }
-
- public TestBolt(TestEnum te){
-
- }
-
- public TestBolt(TestEnum te, float f){
-
- }
-
- public TestBolt(TestEnum te, float f, boolean b){
-
- }
-
- public TestBolt(TestEnum te, float f, boolean b, TestClass... str) {
-
- }
-
- @Override
- public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
- LOG.info("{}", tuple);
- }
-
- @Override
- public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-
- }
-
- // config methods
- public void withFoo(String foo){
- this.foo = foo;
- }
- public void withNone(){
- this.none = "hit";
- }
- public void withBar(String bar){
- this.bar = bar;
- }
-
- public void withFooBar(String foo, String bar){
- this.fooBar = foo + bar;
- }
-
- public void withClasses(TestClass...classes) {
- this.classes = classes;
- }
-
- public String getFoo(){
- return this.foo;
- }
- public String getBar(){
- return this.bar;
- }
-
- public String getFooBar(){
- return this.fooBar;
- }
-
- public TestClass[] getClasses() {
- return classes;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
deleted file mode 100644
index 36b272b..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.kafka.StringScheme;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.operation.BaseFunction;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.operation.builtin.Count;
-import org.apache.storm.trident.testing.FixedBatchSpout;
-import org.apache.storm.trident.testing.MemoryMapState;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-/**
- * Basic Trident example that will return a `StormTopology` from a `getTopology()` method.
- */
-public class TridentTopologySource {
-
- private FixedBatchSpout spout;
-
- public StormTopology getTopology(Config config) {
-
- this.spout = new FixedBatchSpout(new Fields("sentence"), 20,
- new Values("one two"),
- new Values("two three"),
- new Values("three four"),
- new Values("four five"),
- new Values("five six")
- );
-
-
- TridentTopology trident = new TridentTopology();
-
- trident.newStream("wordcount", spout).name("sentence").parallelismHint(1).shuffle()
- .each(new Fields("sentence"), new Split(), new Fields("word"))
- .parallelismHint(1)
- .groupBy(new Fields("word"))
- .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count"))
- .parallelismHint(1);
- return trident.build();
- }
-
- public static class Split extends BaseFunction {
- @Override
- public void execute(TridentTuple tuple, TridentCollector collector) {
- String sentence = tuple.getString(0);
- for (String word : sentence.split(" ")) {
- collector.emit(new Values(word));
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml b/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
deleted file mode 100644
index a29e314..0000000
--- a/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
+++ /dev/null
@@ -1,98 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hbase-wordcount"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-
-components:
- - id: "columnFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- - ["word"]
-
- - id: "counterFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- # !!! the following won't work, and should thow an IllegalArgumentException...
- - "count"
-
- - id: "mapper"
- className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
- configMethods:
- - name: "withRowKeyField"
- args: ["word"]
- - name: "withColumnFields"
- args: [ref: "columnFields"]
- - name: "withCounterFields"
- args: [ref: "counterFields"]
- - name: "withColumnFamily"
- args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- hbase.conf:
- hbase.rootdir: "hdfs://hadoop:54310/hbase"
- hbase.zookeeper.quorum: "hadoop"
-
-# spout definitions
-spouts:
- - id: "word-spout"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-
-bolts:
- - id: "count-bolt"
- className: "org.apache.storm.testing.TestWordCounter"
-
- - id: "hbase-bolt"
- className: "org.apache.storm.hbase.bolt.HBaseBolt"
- constructorArgs:
- - "WordCount" # HBase table name
- - ref: "mapper"
- configMethods:
- - name: "withConfigKey"
- args: ["hbase.conf"]
- parallelism: 1
-
-
-streams:
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "word-spout"
- to: "count-bolt"
- grouping:
- type: SHUFFLE
-
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "count-bolt"
- to: "hbase-bolt"
- grouping:
- type: FIELDS
- args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml b/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
deleted file mode 100644
index 0892ce7..0000000
--- a/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
+++ /dev/null
@@ -1,118 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-# - id: "myComponent"
-# className: "com.foo.bar.MyComponent"
-# constructorArgs:
-# - ...
-# properties:
-# foo: "bar"
-# bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- configMethods:
- - name: "addComponentConfig"
- args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
- - name: "addComponentConfig"
- args:
- - "publisher.data_paths"
- - ["actions", "hello"]
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- configMethods:
- - name: "addComponentConfig"
- args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
- - name: "addComponentConfig"
- args:
- - "publisher.data_paths"
- - ["actions", "hello"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "sentence-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml b/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
deleted file mode 100644
index 7c4ffb3..0000000
--- a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
+++ /dev/null
@@ -1,92 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-name: "yaml-topology"
-
-#
-config:
- topology.workers: 1
- # ...
-
-components:
- - id: "foo"
- className: "org.apache.storm.flux.test.TestBolt$TestClass"
- constructorArgs:
- - "foo"
- - id: "bar"
- className: "org.apache.storm.flux.test.TestBolt$TestClass"
- constructorArgs:
- - "bar"
- - id: "baz"
- className: "org.apache.storm.flux.test.TestBolt$TestClass"
- constructorArgs:
- - "baz"
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.flux.test.TestBolt"
- parallelism: 1
- constructorArgs:
- - FOO # enum class
- - 1.0
- - true
- - reflist: ["foo", "bar"]
- configMethods:
- - name: "withFoo"
- args:
- - "foo"
- - name: "withNone"
- - name: "withBar"
- args:
- - "bar"
- - name: "withFooBar"
- args:
- - "foo"
- - "bar"
- - name: "withClasses"
- args:
- - reflist:
- - "foo"
- - "bar"
- - "baz"
-
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: SHUFFLE
-
-
-
-
-
-
-
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml b/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
deleted file mode 100644
index 957c258..0000000
--- a/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
+++ /dev/null
@@ -1,87 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "diamond-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-bolts:
- - id: "A"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
- - id: "B"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
- - id: "C"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
- - id: "D"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "spout-1"
- to: "A"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - from: "A"
- to: "B"
- grouping:
- type: SHUFFLE
-
- - from: "A"
- to: "C"
- grouping:
- type: SHUFFLE
-
- - from: "C"
- to: "D"
- grouping:
- type: SHUFFLE
-
- - from: "B"
- to: "D"
- grouping:
- type: SHUFFLE
-
-
-
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
deleted file mode 100644
index fceeeed..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
+++ /dev/null
@@ -1,25 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopology"
- methodName: "getTopologyWithDifferentMethodName"
- constructorArgs:
- - "foo"
- - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
deleted file mode 100644
index 440fe4d..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
+++ /dev/null
@@ -1,24 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopologyWithConfigParam"
- constructorArgs:
- - "foo"
- - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
deleted file mode 100644
index 975885b..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
+++ /dev/null
@@ -1,24 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopology"
- constructorArgs:
- - "foo"
- - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
deleted file mode 100644
index 978181b..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
+++ /dev/null
@@ -1,24 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-name: "existing-topology"
-
-config:
- topology.workers: 1
-
-topologySource:
- className: "org.apache.storm.flux.test.TridentTopologySource"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
deleted file mode 100644
index e112c0f..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
+++ /dev/null
@@ -1,23 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopologySource"
- constructorArgs:
- - "foo"
- - "bar"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml b/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
deleted file mode 100644
index 2bccb33..0000000
--- a/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
+++ /dev/null
@@ -1,97 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hdfs-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
- - id: "syncPolicy"
- className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
- constructorArgs:
- - 1000
- - id: "rotationPolicy"
- className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
- constructorArgs:
- - 5.0
- - MB
-
- - id: "fileNameFormat"
- className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
- configMethods:
- - name: "withPath"
- args: ["/tmp/foo/"]
- - name: "withExtension"
- args: [".txt"]
-
- - id: "recordFormat"
- className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
- configMethods:
- - name: "withFieldDelimiter"
- args: ["|"]
-
- - id: "rotationAction"
- className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
- configMethods:
- - name: "toDestination"
- args: ["/tmp/dest2"]
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
- # ...
-
-# bolt definitions
-
-# HdfsBolt bolt = new HdfsBolt()
-# .withConfigKey("hdfs.config")
-# .withFsUrl(args[0])
-# .withFileNameFormat(fileNameFormat)
-# .withRecordFormat(format)
-# .withRotationPolicy(rotationPolicy)
-# .withSyncPolicy(syncPolicy)
-# .addRotationAction(new MoveFileAction().toDestination("/tmp/dest2/"));
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.hdfs.bolt.HdfsBolt"
- configMethods:
- - name: "withConfigKey"
- args: ["hdfs.config"]
- - name: "withFsUrl"
- args: ["hdfs://hadoop:54310"]
- - name: "withFileNameFormat"
- args: [ref: "fileNameFormat"]
- - name: "withRecordFormat"
- args: [ref: "recordFormat"]
- - name: "withRotationPolicy"
- args: [ref: "rotationPolicy"]
- - name: "withSyncPolicy"
- args: [ref: "syncPolicy"]
- - name: "addRotationAction"
- args: [ref: "rotationAction"]
- parallelism: 1
- # ...
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/include_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/include_test.yaml b/external/flux/flux-core/src/test/resources/configs/include_test.yaml
deleted file mode 100644
index 702f590..0000000
--- a/external/flux/flux-core/src/test/resources/configs/include_test.yaml
+++ /dev/null
@@ -1,25 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test includes by defining nothing, and simply override the topology name
----
-
-name: "include-topology"
-
-includes:
- - resource: true
- file: "/configs/shell_test.yaml"
- override: false #otherwise subsequent includes that define 'name' would override
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml b/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
deleted file mode 100644
index c2dfac0..0000000
--- a/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
+++ /dev/null
@@ -1,33 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# This is an invalid config. It defines both a topologySource and a list of spouts.
----
-
-name: "existing-topology"
-topologySource:
- className: "org.apache.storm.flux.test.SimpleTopologySource"
-
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- parallelism: 1
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml b/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
deleted file mode 100644
index 1fb59ca..0000000
--- a/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
+++ /dev/null
@@ -1,126 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-# topology definition
-# name to be used when submitting
-name: "kafka-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
- - id: "stringScheme"
- className: "org.apache.storm.kafka.StringScheme"
-
- - id: "stringMultiScheme"
- className: "org.apache.storm.spout.SchemeAsMultiScheme"
- constructorArgs:
- - ref: "stringScheme"
-
- - id: "zkHosts"
- className: "org.apache.storm.kafka.ZkHosts"
- constructorArgs:
- - "localhost:2181"
-
-# Alternative kafka config
-# - id: "kafkaConfig"
-# className: "org.apache.storm.kafka.KafkaConfig"
-# constructorArgs:
-# # brokerHosts
-# - ref: "zkHosts"
-# # topic
-# - "myKafkaTopic"
-# # clientId (optional)
-# - "myKafkaClientId"
-
- - id: "spoutConfig"
- className: "org.apache.storm.kafka.SpoutConfig"
- constructorArgs:
- # brokerHosts
- - ref: "zkHosts"
- # topic
- - "myKafkaTopic"
- # zkRoot
- - "/kafkaSpout"
- # id
- - "myId"
- properties:
- - name: "ignoreZkOffsets"
- value: true
- - name: "scheme"
- ref: "stringMultiScheme"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "kafka-spout"
- className: "org.apache.storm.kafka.KafkaSpout"
- constructorArgs:
- - ref: "spoutConfig"
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "kafka-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/shell_test.yaml b/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
deleted file mode 100644
index dfab397..0000000
--- a/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
+++ /dev/null
@@ -1,118 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-# - id: "myComponent"
-# className: "com.foo.bar.MyComponent"
-# constructorArgs:
-# - ...
-# properties:
-# foo: "bar"
-# bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- configMethods:
- - name: "addComponentConfig"
- args: ["rabbitmq.configfile", "etc/rabbit.yml"]
- - name: "addComponentConfig"
- args:
- - "publisher.data_paths"
- - ["actions"]
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- configMethods:
- - name: "addComponentConfig"
- args: ["rabbitmq.configfile", "etc/rabbit.yml"]
- - name: "addComponentConfig"
- args:
- - "publisher.data_paths"
- - ["actions"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "sentence-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml b/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
deleted file mode 100644
index b841b53..0000000
--- a/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
+++ /dev/null
@@ -1,120 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hbase-wordcount"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-
-# WordSpout spout = new WordSpout();
-# WordCounter bolt = new WordCounter();
-#
-# SimpleHBaseMapper mapper = new SimpleHBaseMapper()
-# .withRowKeyField("word")
-# .withColumnFields(new Fields("word"))
-# .withCounterFields(new Fields("count"))
-# .withColumnFamily("cf");
-#
-# HBaseBolt hbase = new HBaseBolt("WordCount", mapper)
-# .withConfigKey("hbase.conf");
-#
-#
-# // wordSpout ==> countBolt ==> HBaseBolt
-# TopologyBuilder builder = new TopologyBuilder();
-#
-# builder.setSpout(WORD_SPOUT, spout, 1);
-# builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
-# builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
-
-
-
-
-components:
- - id: "columnFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- - ["word"]
-
- - id: "counterFields"
- className: "org.apache.storm.tuple.Fields"
- constructorArgs:
- - ["count"]
-
- - id: "mapper"
- className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
- configMethods:
- - name: "withRowKeyField"
- args: ["word"]
- - name: "withColumnFields"
- args: [ref: "columnFields"]
- - name: "withCounterFields"
- args: [ref: "counterFields"]
- - name: "withColumnFamily"
- args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- hbase.conf:
- hbase.rootdir: "hdfs://hadoop:54310/hbase"
- hbase.zookeeper.quorum: "hadoop"
-
-# spout definitions
-spouts:
- - id: "word-spout"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
-
-# bolt definitions
-
-bolts:
- - id: "count-bolt"
- className: "org.apache.storm.testing.TestWordCounter"
-
- - id: "hbase-bolt"
- className: "org.apache.storm.hbase.bolt.HBaseBolt"
- constructorArgs:
- - "WordCount" # HBase table name
- - ref: "mapper"
- configMethods:
- - name: "withConfigKey"
- args: ["hbase.conf"]
- parallelism: 1
-
-
-streams:
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "word-spout"
- to: "count-bolt"
- grouping:
- type: SHUFFLE
-
- - name: "" # name isn't used (placeholder for logging, UI, etc.)
- from: "count-bolt"
- to: "hbase-bolt"
- grouping:
- type: FIELDS
- args: ["word"]
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml b/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
deleted file mode 100644
index ce9e62d..0000000
--- a/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
+++ /dev/null
@@ -1,106 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "${topology.name}"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-# - id: "myComponent"
-# className: "com.foo.bar.MyComponent"
-# constructorArgs:
-# - ...
-# properties:
-# foo: "bar"
-# bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # test environent variable substitution
- test.env.value: ${ENV-PATH}
- # ...
-
-# spout definitions
-spouts:
- - id: "sentence-spout"
- className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
- # shell spout constructor takes 2 arguments: String[], String[]
- constructorArgs:
- # command line
- - ["node", "randomsentence.js"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "splitsentence"
- className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
- constructorArgs:
- # command line
- - ["python", "splitsentence.py"]
- # output fields
- - ["word"]
- parallelism: 1
- # ...
-
- - id: "log"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
- - id: "count"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
- - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
- from: "sentence-spout"
- to: "splitsentence"
- grouping:
- type: SHUFFLE
-
- - name: "split --> count"
- from: "splitsentence"
- to: "count"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "count --> log"
- from: "count"
- to: "log"
- grouping:
- type: SHUFFLE
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/tck.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/tck.yaml b/external/flux/flux-core/src/test/resources/configs/tck.yaml
deleted file mode 100644
index 5d40445..0000000
--- a/external/flux/flux-core/src/test/resources/configs/tck.yaml
+++ /dev/null
@@ -1,95 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-
-# YAML configuration to serve as a basic smoke test for what is supported.
-#
-# We should support comments, so if we've failed so far, things aren't good.
-
-# we shouldn't choke if we see a document separator...
----
-
-# topology definition
-# name to be used when submitting
-name: "yaml-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-# - id: "myComponent"
-# className: "com.foo.bar.MyComponent"
-# properties:
-# foo: "bar"
-# bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
- topology.workers: 1
- # ...
-
-# spout definitions
-spouts:
- - id: "spout-1"
- className: "org.apache.storm.testing.TestWordSpout"
- parallelism: 1
- # ...
-
-# bolt definitions
-bolts:
- - id: "bolt-1"
- className: "org.apache.storm.testing.TestWordCounter"
- parallelism: 1
- # ...
-
- - id: "bolt-2"
- className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
- parallelism: 1
- # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
- - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-# id: "connection-1"
- from: "spout-1"
- to: "bolt-1"
- grouping:
- type: FIELDS
- args: ["word"]
-
- - name: "bolt-1 --> bolt2"
- from: "bolt-1"
- to: "bolt-2"
- grouping:
- type: CUSTOM
- customClass:
- className: "org.apache.storm.testing.NGrouping"
- constructorArgs:
- - 1
-
-
-
-
-
-
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/test.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/test.properties b/external/flux/flux-core/src/test/resources/configs/test.properties
deleted file mode 100644
index ecd89d9..0000000
--- a/external/flux/flux-core/src/test/resources/configs/test.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-topology.name: substitution-topology
-some.other.property: foo bar
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/log4j2.xml b/external/flux/flux-core/src/test/resources/log4j2.xml
deleted file mode 100644
index d3333f3..0000000
--- a/external/flux/flux-core/src/test/resources/log4j2.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<Configuration scan="true" monitorInterval="30">
- <Appenders>
- <Console name="A1" target="SYSTEM_OUT">
- <PatternLayout>
- <pattern>%-4r [%t] %-5level %c{1} - %msg%n</pattern>
- </PatternLayout>
- </Console>
- </Appenders>
- <Loggers>
- <Logger name="org.apache.storm.curator" level="warn"/>
- <Logger name="org.apache.storm.flux" level="debug"/>
- <Logger name="org.apache.storm.zookeeper" level="warn"/>
- <Root level="debug">
- <appender-ref ref="A1"/>
- </Root>
- </Loggers>
-</Configuration>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/README.md
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/README.md b/external/flux/flux-examples/README.md
deleted file mode 100644
index 3d610b4..0000000
--- a/external/flux/flux-examples/README.md
+++ /dev/null
@@ -1,93 +0,0 @@
-# Flux Examples
-A collection of examples illustrating various capabilities.
-
-## Building From Source and Running
-
-Checkout the projects source and perform a top level Maven build (i.e. from the `flux` directory):
-
-```bash
-git clone https://github.com/apache/storm.git
-cd storm
-mvn install -DskipTests=true
-```
-
-This will create a shaded (i.e. "fat" or "uber") jar in the `external/flux/flux-examples/target` directory that can run/deployed with
-the `storm` command:
-
-```bash
-cd flux-examples
-storm jar ./target/flux-examples-*-SNAPSHOT.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_wordcount.yaml
-```
-
-The example YAML files are also packaged in the examples jar, so they can also be referenced with Flux's `--resource`
-command line switch:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local --resource /simple_wordcount.yaml
-```
-
-## Available Examples
-
-### [simple_wordcount.yaml](src/main/resources/simple_wordcount.yaml)
-
-This is a very basic wordcount example using Java spouts and bolts. It simply logs the running count of each word
-received.
-
-### [multilang.yaml](src/main/resources/multilang.yaml)
-
-Another wordcount example that uses a spout written in JavaScript (node.js), a bolt written in Python, and two bolts
-written in java.
-
-### [kafka_spout.yaml](src/main/resources/kafka_spout.yaml)
-
-This example illustrates how to configure Storm's `storm-kafka` spout using Flux YAML DSL `components`, `references`,
-and `constructor arguments` constructs.
-
-### [simple_hdfs.yaml](src/main/resources/simple_hdfs.yaml)
-
-This example demonstrates using Flux to setup a storm-hdfs bolt to write to an HDFS cluster. It also demonstrates Flux's
-variable substitution/filtering feature.
-
-To run the `simple_hdfs.yaml` example, copy the `hdfs_bolt.properties` file to a convenient location and change, at
-least, the property `hdfs.url` to point to a HDFS cluster. Then you can run the example something like:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hdfs.yaml --filter my_hdfs_bolt.properties
-```
-
-### [simple_hbase.yaml](src/main/resources/simple_hbase.yaml)
-
-This example illustrates how to use Flux to setup a storm-hbase bolt to write to HBase.
-
-To run the `simple_hbase.yaml` example, copy the `hbase_bolt.properties` file to a convenient location and change the properties
- `hbase.rootdir` and `hbase.zookeeper.quorum`. Then you can run the example something like:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hbase.yaml --filter my_hbase_bolt.properties
-```
-
-### [simple_windowing.yaml](src/main/resources/simple_windowing.yaml)
-
-This example illustrates how to use Flux to set up a storm topology that contains windowing operations.
-
-To run,
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_windowing.yaml
-```
-
-### [simple_stateful_wordcount.yaml](src/main/resources/simple_stateful_wordcount.yaml)
-
-Flux also supports stateful bolts which is illustrated with this example. It is basically an extension of the basic wordcount example.
-The state is periodically saved (checkpointed) and restored when the topology is restarted.
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
-```
-
-By default the state is stored in-memory only. As such you won't see a resumed state unless you configure to use Redis as the state backend.
-Ensure that you have Redis running at `localhost:6379` and that `storm-redis-*.jar` is in the classpath.
-
-```bash
-STORM_EXT_CLASSPATH=../../storm-redis/target storm jar ./target/flux-examples-*.jar -c topology.state.provider=org.apache.storm.redis.state.RedisKeyValueStateProvider org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
-```
[05/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
new file mode 100644
index 0000000..437877c
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner;
+
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+public class StormRelDataTypeSystem extends RelDataTypeSystemImpl {
+ public static final RelDataTypeSystem STORM_REL_DATATYPE_SYSTEM = new StormRelDataTypeSystem();
+
+ @Override
+ public int getMaxNumericScale() {
+ return 38;
+ }
+
+ @Override
+ public int getMaxNumericPrecision() {
+ return 38;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
new file mode 100644
index 0000000..40bbacd
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class StormRelUtils {
+ private static final Logger LOG = LoggerFactory.getLogger(StormRelUtils.class);
+
+ private static final AtomicInteger sequence = new AtomicInteger(0);
+ private static final AtomicInteger classSequence = new AtomicInteger(0);
+
+ public static String getStageName(TridentRel relNode) {
+ return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" + sequence.getAndIncrement();
+ }
+
+ public static String getClassName(TridentRel relNode) {
+ return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" +
+ classSequence.getAndIncrement();
+ }
+
+ public static TridentRel getStormRelInput(RelNode input) {
+ if (input instanceof RelSubset) {
+ // go with known best input
+ input = ((RelSubset) input).getBest();
+ }
+ return (TridentRel) input;
+ }
+
+ public static String explain(final RelNode rel) {
+ return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
+ }
+
+ public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
+ String explain = "";
+ try {
+ explain = RelOptUtil.toString(rel);
+ } catch (StackOverflowError e) {
+ LOG.error("StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list.");
+ LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
+ LOG.error("Forcing plan to empty string and continue... SQL Runner may not working properly after.");
+ }
+ return explain;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
new file mode 100644
index 0000000..258fe72
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner;
+
+import org.apache.calcite.sql.util.SqlShuttle;
+
+public class UnsupportedOperatorsVisitor extends SqlShuttle {
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
new file mode 100644
index 0000000..29ca08b
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rex.RexProgram;
+
+public abstract class StormCalcRelBase extends Calc implements StormRelNode {
+ protected StormCalcRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
+ super(cluster, traits, child, program);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
new file mode 100644
index 0000000..fa2ac65
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+public abstract class StormFilterRelBase extends Filter implements StormRelNode {
+ protected StormFilterRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+ super(cluster, traits, child, condition);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
new file mode 100644
index 0000000..d8e82c5
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.Set;
+
+public abstract class StormJoinRelBase extends Join implements StormRelNode {
+ protected StormJoinRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+ super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
new file mode 100644
index 0000000..fe32ba5
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+public abstract class StormProjectRelBase extends Project implements StormRelNode {
+ protected StormProjectRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+ super(cluster, traits, input, projects, rowType);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
new file mode 100644
index 0000000..9327868
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.rel.RelNode;
+
+public interface StormRelNode extends RelNode {
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
new file mode 100644
index 0000000..59694fc
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+public abstract class StormStreamInsertRelBase extends TableModify implements StormRelNode {
+ protected StormStreamInsertRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+ super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
new file mode 100644
index 0000000..32f1ac2
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+
+public abstract class StormStreamScanRelBase extends TableScan implements StormRelNode {
+
+ // FIXME: define Table class and table.unwrap() to get it
+
+ protected StormStreamScanRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+ super(cluster, traitSet, table);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
new file mode 100644
index 0000000..f98fb02
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
@@ -0,0 +1,156 @@
+/*
+ *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements. See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership. The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License. You may obtain a copy of the License at
+ * *
+ * * http://www.apache.org/licenses/LICENSE-2.0
+ * *
+ * * Unless required by applicable law or agreed to in writing, software
+ * * distributed under the License is distributed on an "AS IS" BASIS,
+ * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * * See the License for the specific language governing permissions and
+ * * limitations under the License.
+ *
+ */
+package org.apache.storm.sql.planner.trident;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.planner.StormRelDataTypeSystem;
+import org.apache.storm.sql.planner.UnsupportedOperatorsVisitor;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.AbstractTridentProcessor;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class QueryPlanner {
+
+ public static final int STORM_REL_CONVERSION_RULES = 1;
+
+ private final Planner planner;
+
+ private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+ RelDataTypeSystem.DEFAULT);
+
+ public QueryPlanner(SchemaPlus schema) {
+ final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+
+ traitDefs.add(ConventionTraitDef.INSTANCE);
+ traitDefs.add(RelCollationTraitDef.INSTANCE);
+
+ List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+ sqlOperatorTables.add(SqlStdOperatorTable.instance());
+ sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+ false,
+ Collections.<String>emptyList(), typeFactory));
+
+ FrameworkConfig config = Frameworks.newConfigBuilder()
+ .defaultSchema(schema)
+ .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
+ .traitDefs(traitDefs)
+ .context(Contexts.EMPTY_CONTEXT)
+ .ruleSets(TridentStormRuleSets.getRuleSets())
+ .costFactory(null)
+ .typeSystem(StormRelDataTypeSystem.STORM_REL_DATATYPE_SYSTEM)
+ .build();
+ this.planner = Frameworks.getPlanner(config);
+ }
+
+ public AbstractTridentProcessor compile(Map<String, ISqlTridentDataSource> sources, String query) throws Exception {
+ TridentRel relNode = getPlan(query);
+
+ TridentPlanCreator tridentPlanCreator = new TridentPlanCreator(sources, new RexBuilder(typeFactory));
+ relNode.tridentPlan(tridentPlanCreator);
+
+ final TridentTopology topology = tridentPlanCreator.getTopology();
+ final IAggregatableStream lastStream = tridentPlanCreator.pop();
+ final DataContext dc = tridentPlanCreator.getDataContext();
+ final List<CompilingClassLoader> cls = tridentPlanCreator.getClassLoaders();
+
+ return new AbstractTridentProcessor() {
+ @Override
+ public TridentTopology build() {
+ return topology;
+ }
+
+ @Override
+ public Stream outputStream() {
+ return lastStream.toStream();
+ }
+
+ @Override
+ public DataContext getDataContext() {
+ return dc;
+ }
+
+ @Override
+ public List<CompilingClassLoader> getClassLoaders() {
+ return cls;
+ }
+ };
+ }
+
+ public TridentRel getPlan(String query) throws ValidationException, RelConversionException, SqlParseException {
+ return (TridentRel) validateAndConvert(planner.parse(query));
+ }
+
+ private RelNode validateAndConvert(SqlNode sqlNode) throws ValidationException, RelConversionException {
+ SqlNode validated = validateNode(sqlNode);
+ RelNode relNode = convertToRelNode(validated);
+ return convertToStormRel(relNode);
+ }
+
+ private RelNode convertToStormRel(RelNode relNode) throws RelConversionException {
+ RelTraitSet traitSet = relNode.getTraitSet();
+ traitSet = traitSet.simplify();
+
+ // PlannerImpl.transform() optimizes RelNode with ruleset
+ return planner.transform(STORM_REL_CONVERSION_RULES, traitSet.plus(TridentLogicalConvention.INSTANCE), relNode);
+ }
+
+ private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
+ return planner.rel(sqlNode).rel;
+ }
+
+ private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
+ SqlNode validatedSqlNode = planner.validate(sqlNode);
+ validatedSqlNode.accept(new UnsupportedOperatorsVisitor());
+ return validatedSqlNode;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
new file mode 100644
index 0000000..30ebf7e
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.StormDataContext;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+
+public class TridentPlanCreator {
+ private final Map<String, ISqlTridentDataSource> sources;
+ private final JavaTypeFactory typeFactory;
+ private final RexNodeToJavaCodeCompiler rexCompiler;
+ private final DataContext dataContext;
+ private final TridentTopology topology;
+
+ private final Deque<IAggregatableStream> streamStack = new ArrayDeque<>();
+ private final List<CompilingClassLoader> classLoaders = new ArrayList<>();
+
+ public TridentPlanCreator(Map<String, ISqlTridentDataSource> sources, RexBuilder rexBuilder) {
+ this.sources = sources;
+ this.rexCompiler = new RexNodeToJavaCodeCompiler(rexBuilder);
+ this.typeFactory = (JavaTypeFactory) rexBuilder.getTypeFactory();
+
+ this.topology = new TridentTopology();
+ this.dataContext = new StormDataContext();
+ }
+
+ public void addStream(IAggregatableStream stream) throws Exception {
+ push(stream);
+ }
+
+ public IAggregatableStream pop() {
+ return streamStack.pop();
+ }
+
+ public Map<String, ISqlTridentDataSource> getSources() {
+ return sources;
+ }
+
+ public DataContext getDataContext() {
+ return dataContext;
+ }
+
+ public JavaTypeFactory getTypeFactory() {
+ return typeFactory;
+ }
+
+ public TridentTopology getTopology() {
+ return topology;
+ }
+
+ public ExecutableExpression createScalarInstance(List<RexNode> nodes, RelDataType inputRowType, String className)
+ throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
+ String expr = rexCompiler.compile(nodes, inputRowType, className);
+ CompilingClassLoader classLoader = new CompilingClassLoader(
+ getLastClassLoader(), className, expr, null);
+ ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
+ addClassLoader(classLoader);
+ return new DebuggableExecutableExpression(instance, expr);
+ }
+
+ public ExecutableExpression createScalarInstance(RexProgram program, String className)
+ throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
+ String expr = rexCompiler.compile(program, className);
+ CompilingClassLoader classLoader = new CompilingClassLoader(
+ getLastClassLoader(), className, expr, null);
+ ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
+ addClassLoader(classLoader);
+ return new DebuggableExecutableExpression(instance, expr);
+ }
+
+ private void push(IAggregatableStream stream) {
+ streamStack.push(stream);
+ }
+
+ public void addClassLoader(CompilingClassLoader compilingClassLoader) {
+ this.classLoaders.add(compilingClassLoader);
+ }
+
+ public ClassLoader getLastClassLoader() {
+ if (this.classLoaders.size() > 0) {
+ return this.classLoaders.get(this.classLoaders.size() - 1);
+ } else {
+ return this.getClass().getClassLoader();
+ }
+ }
+
+ public List<CompilingClassLoader> getClassLoaders() {
+ return classLoaders;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
new file mode 100644
index 0000000..e146069
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
@@ -0,0 +1,110 @@
+/*
+ *
+ * * Licensed to the Apache Software Foundation (ASF) under one
+ * * or more contributor license agreements. See the NOTICE file
+ * * distributed with this work for additional information
+ * * regarding copyright ownership. The ASF licenses this file
+ * * to you under the Apache License, Version 2.0 (the
+ * * "License"); you may not use this file except in compliance
+ * * with the License. You may obtain a copy of the License at
+ * *
+ * * http://www.apache.org/licenses/LICENSE-2.0
+ * *
+ * * Unless required by applicable law or agreed to in writing, software
+ * * distributed under the License is distributed on an "AS IS" BASIS,
+ * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * * See the License for the specific language governing permissions and
+ * * limitations under the License.
+ *
+ */
+package org.apache.storm.sql.planner.trident;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.UnionEliminatorRule;
+import org.apache.calcite.rel.stream.StreamRules;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.storm.sql.planner.trident.rules.TridentCalcRule;
+import org.apache.storm.sql.planner.trident.rules.TridentFilterRule;
+import org.apache.storm.sql.planner.trident.rules.TridentScanRule;
+import org.apache.storm.sql.planner.trident.rules.TridentAggregateRule;
+import org.apache.storm.sql.planner.trident.rules.TridentJoinRule;
+import org.apache.storm.sql.planner.trident.rules.TridentModifyRule;
+import org.apache.storm.sql.planner.trident.rules.TridentProjectRule;
+
+import java.util.Iterator;
+
+public class TridentStormRuleSets {
+ private static final ImmutableSet<RelOptRule> calciteToStormConversionRules =
+ ImmutableSet.<RelOptRule>builder().add(
+ SortRemoveRule.INSTANCE,
+
+ FilterToCalcRule.INSTANCE,
+ ProjectToCalcRule.INSTANCE,
+ FilterCalcMergeRule.INSTANCE,
+ ProjectCalcMergeRule.INSTANCE,
+ CalcMergeRule.INSTANCE,
+
+ PruneEmptyRules.FILTER_INSTANCE,
+ PruneEmptyRules.PROJECT_INSTANCE,
+ PruneEmptyRules.UNION_INSTANCE,
+
+ ProjectFilterTransposeRule.INSTANCE,
+ FilterProjectTransposeRule.INSTANCE,
+ ProjectRemoveRule.INSTANCE,
+
+ ReduceExpressionsRule.FILTER_INSTANCE,
+ ReduceExpressionsRule.PROJECT_INSTANCE,
+ ReduceExpressionsRule.CALC_INSTANCE,
+
+ // merge and push unions rules
+ UnionEliminatorRule.INSTANCE,
+
+ TridentScanRule.INSTANCE,
+ TridentFilterRule.INSTANCE,
+ TridentProjectRule.INSTANCE,
+ TridentAggregateRule.INSTANCE,
+ TridentJoinRule.INSTANCE,
+ TridentModifyRule.INSTANCE,
+ TridentCalcRule.INSTANCE
+ ).build();
+
+ public static RuleSet[] getRuleSets() {
+ return new RuleSet[]{
+ new StormRuleSet(StreamRules.RULES),
+ new StormRuleSet(ImmutableSet.<RelOptRule>builder().addAll(StreamRules.RULES).addAll(calciteToStormConversionRules).build())
+ };
+ }
+
+ private static class StormRuleSet implements RuleSet {
+ final ImmutableSet<RelOptRule> rules;
+
+ public StormRuleSet(ImmutableSet<RelOptRule> rules) {
+ this.rules = rules;
+ }
+
+ public StormRuleSet(ImmutableList<RelOptRule> rules) {
+ this.rules = ImmutableSet.<RelOptRule>builder()
+ .addAll(rules)
+ .build();
+ }
+
+ @Override
+ public Iterator<RelOptRule> iterator() {
+ return rules.iterator();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
new file mode 100644
index 0000000..482e841
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormCalcRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationCalc;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TridentCalcRel extends StormCalcRelBase implements TridentRel {
+ public TridentCalcRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
+ super(cluster, traits, child, program);
+ }
+
+ @Override
+ public Calc copy(RelTraitSet traitSet, RelNode child, RexProgram program) {
+ return new TridentCalcRel(getCluster(), traitSet, child, program);
+ }
+
+ @Override
+ public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+ // SingleRel
+ RelNode input = getInput();
+ StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+ Stream inputStream = planCreator.pop().toStream();
+
+ String stageName = StormRelUtils.getStageName(this);
+
+ RelDataType inputRowType = getInput(0).getRowType();
+
+ List<String> outputFieldNames = getRowType().getFieldNames();
+ int outputCount = outputFieldNames.size();
+
+ // filter
+ ExecutableExpression filterInstance = null;
+ RexLocalRef condition = program.getCondition();
+ if (condition != null) {
+ RexNode conditionNode = program.expandLocalRef(condition);
+ filterInstance = planCreator.createScalarInstance(Lists.newArrayList(conditionNode), inputRowType,
+ StormRelUtils.getClassName(this));
+ }
+
+ // projection
+ ExecutableExpression projectionInstance = null;
+ List<RexLocalRef> projectList = program.getProjectList();
+ if (projectList != null && !projectList.isEmpty()) {
+ List<RexNode> expandedNodes = new ArrayList<>();
+ for (RexLocalRef project : projectList) {
+ expandedNodes.add(program.expandLocalRef(project));
+ }
+
+ projectionInstance = planCreator.createScalarInstance(expandedNodes, inputRowType,
+ StormRelUtils.getClassName(this));
+ }
+
+ if (projectionInstance == null && filterInstance == null) {
+ // it shouldn't be happen
+ throw new IllegalStateException("Either projection or condition, or both should be provided.");
+ }
+
+ final Stream finalStream = inputStream
+ .flatMap(new EvaluationCalc(filterInstance, projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
+ .name(stageName);
+
+ planCreator.addStream(finalStream);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
new file mode 100644
index 0000000..1fe0927
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormFilterRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationFilter;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.List;
+
+public class TridentFilterRel extends StormFilterRelBase implements TridentRel {
+ public TridentFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+ super(cluster, traits, child, condition);
+ }
+
+ @Override
+ public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+ return new TridentFilterRel(getCluster(), traitSet, input, condition);
+ }
+
+ @Override
+ public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+ // SingleRel
+ RelNode input = getInput();
+ StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+ Stream inputStream = planCreator.pop().toStream();
+
+ String stageName = StormRelUtils.getStageName(this);
+
+ List<RexNode> childExps = getChildExps();
+ RelDataType inputRowType = getInput(0).getRowType();
+
+ String filterClassName = StormRelUtils.getClassName(this);
+ ExecutableExpression filterInstance = planCreator.createScalarInstance(childExps, inputRowType, filterClassName);
+
+ IAggregatableStream finalStream = inputStream.filter(new EvaluationFilter(filterInstance, planCreator.getDataContext()))
+ .name(stageName);
+ planCreator.addStream(finalStream);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
new file mode 100644
index 0000000..d221498
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+
+public enum TridentLogicalConvention implements Convention {
+ INSTANCE;
+
+ @Override
+ public Class getInterface() {
+ return TridentRel.class;
+ }
+
+ @Override
+ public String getName() {
+ return "STORM_LOGICAL";
+ }
+
+ @Override
+ public RelTraitDef getTraitDef() {
+ return ConventionTraitDef.INSTANCE;
+ }
+
+ @Override
+ public boolean satisfies(RelTrait trait) {
+ return this == trait;
+ }
+
+ @Override
+ public void register(RelOptPlanner planner) {}
+
+ @Override
+ public String toString() {
+ return getName();
+ }
+
+ @Override
+ public boolean canConvertConvention(Convention toConvention) {
+ return false;
+ }
+
+ @Override
+ public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+ return false;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
new file mode 100644
index 0000000..06be5d7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormProjectRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationFunction;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+
+public class TridentProjectRel extends StormProjectRelBase implements TridentRel {
+ public TridentProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+ super(cluster, traits, input, projects, rowType);
+ }
+
+ @Override
+ public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
+ return new TridentProjectRel(getCluster(), traitSet, input, projects, rowType);
+ }
+
+ @Override
+ public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+ // SingleRel
+ RelNode input = getInput();
+ StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+ Stream inputStream = planCreator.pop().toStream();
+
+ String stageName = StormRelUtils.getStageName(this);
+ String projectionClassName = StormRelUtils.getClassName(this);
+
+ List<String> outputFieldNames = getRowType().getFieldNames();
+ int outputCount = outputFieldNames.size();
+
+ List<RexNode> childExps = getChildExps();
+ RelDataType inputRowType = getInput(0).getRowType();
+
+ ExecutableExpression projectionInstance = planCreator.createScalarInstance(childExps, inputRowType, projectionClassName);
+ Stream finalStream = inputStream
+ .map(new EvaluationFunction(projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
+ .name(stageName);
+
+ planCreator.addStream(finalStream);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
new file mode 100644
index 0000000..8b8e949
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import org.apache.storm.sql.planner.rel.StormRelNode;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+
+public interface TridentRel extends StormRelNode {
+ void tridentPlan(TridentPlanCreator planCreator) throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
new file mode 100644
index 0000000..e92c29b
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormStreamInsertRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+
+public class TridentStreamInsertRel extends StormStreamInsertRelBase implements TridentRel {
+ public TridentStreamInsertRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+ super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
+ }
+
+ @Override
+ public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ return new TridentStreamInsertRel(getCluster(), traitSet, getTable(), getCatalogReader(),
+ sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
+ }
+
+ @Override
+ public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+ // SingleRel
+ RelNode input = getInput();
+ StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+ Stream inputStream = planCreator.pop().toStream();
+
+ String stageName = StormRelUtils.getStageName(this);
+
+ Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported.");
+
+ List<String> inputFields = this.input.getRowType().getFieldNames();
+ List<String> outputFields = getRowType().getFieldNames();
+
+ // FIXME: this should be really different...
+ String tableName = Joiner.on('.').join(getTable().getQualifiedName());
+ ISqlTridentDataSource.SqlTridentConsumer consumer = planCreator.getSources().get(tableName).getConsumer();
+
+ // In fact this is normally the end of stream, but I'm still not sure so I open new streams based on State values
+ IAggregatableStream finalStream = inputStream
+ .partitionPersist(consumer.getStateFactory(), new Fields(inputFields), consumer.getStateUpdater(),
+ new Fields(outputFields))
+ .newValuesStream().name(stageName);
+
+ planCreator.addStream(finalStream);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
new file mode 100644
index 0000000..c563d73
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rel;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormStreamScanRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.Map;
+
+public class TridentStreamScanRel extends StormStreamScanRelBase implements TridentRel {
+ private final int parallelismHint;
+
+ public TridentStreamScanRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table, int parallelismHint) {
+ super(cluster, traitSet, table);
+ this.parallelismHint = parallelismHint;
+ }
+
+ @Override
+ public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+ String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+ // FIXME: this should be really different...
+ Map<String, ISqlTridentDataSource> sources = planCreator.getSources();
+ if (!sources.containsKey(sourceName)) {
+ throw new RuntimeException("Cannot find table " + sourceName);
+ }
+
+ String stageName = StormRelUtils.getStageName(this);
+ IAggregatableStream finalStream = planCreator.getTopology().newStream(stageName, sources.get(sourceName).getProducer())
+ .parallelismHint(parallelismHint);
+ planCreator.addStream(finalStream);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
new file mode 100644
index 0000000..ac35414
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentAggregateRule extends ConverterRule {
+ public static final RelOptRule INSTANCE = new TridentAggregateRule();
+
+ private TridentAggregateRule() {
+ super(LogicalAggregate.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentAggregateRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ throw new UnsupportedOperationException("Aggregate operation is not supported.");
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
new file mode 100644
index 0000000..25126ec
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.storm.sql.planner.trident.rel.TridentCalcRel;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentCalcRule extends ConverterRule {
+ public static final TridentCalcRule INSTANCE = new TridentCalcRule();
+
+ private TridentCalcRule() {
+ super(LogicalCalc.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentCalcRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final Calc calc = (Calc) rel;
+ final RelNode input = calc.getInput();
+
+ return new TridentCalcRel(calc.getCluster(), calc.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+ convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
+ calc.getProgram());
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
new file mode 100644
index 0000000..7f9c41f
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.storm.sql.planner.trident.rel.TridentFilterRel;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentFilterRule extends ConverterRule {
+ public static TridentFilterRule INSTANCE = new TridentFilterRule();
+
+ private TridentFilterRule() {
+ super(LogicalFilter.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentFilterRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final Filter filter = (Filter) rel;
+ final RelNode input = filter.getInput();
+
+ return new TridentFilterRel(filter.getCluster(),
+ filter.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+ convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
+ filter.getCondition());
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
new file mode 100644
index 0000000..90f5083
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentJoinRule extends ConverterRule {
+ public static final TridentJoinRule INSTANCE = new TridentJoinRule();
+
+ private TridentJoinRule() {
+ super(LogicalJoin.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentJoinRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ throw new UnsupportedOperationException("Join operation is not supported.");
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
new file mode 100644
index 0000000..2155451
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Table;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentStreamInsertRel;
+
+import java.util.List;
+
+public class TridentModifyRule extends ConverterRule {
+ public static final TridentModifyRule INSTANCE = new TridentModifyRule();
+
+ private TridentModifyRule() {
+ super(LogicalTableModify.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentModifyRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final TableModify tableModify = (TableModify) rel;
+ final RelNode input = tableModify.getInput();
+
+ final RelOptCluster cluster = tableModify.getCluster();
+ final RelTraitSet traitSet = tableModify.getTraitSet().replace(TridentLogicalConvention.INSTANCE);
+ final RelOptTable relOptTable = tableModify.getTable();
+ final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
+ final RelNode convertedInput = convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE));
+ final TableModify.Operation operation = tableModify.getOperation();
+ final List<String> updateColumnList = tableModify.getUpdateColumnList();
+ final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
+ final boolean flattened = tableModify.isFlattened();
+
+ final Table table = tableModify.getTable().unwrap(Table.class);
+
+ switch (table.getJdbcTableType()) {
+ case STREAM:
+ if (operation != TableModify.Operation.INSERT) {
+ throw new UnsupportedOperationException(String.format("Streams doesn't support %s modify operation", operation));
+ }
+ return new TridentStreamInsertRel(cluster, traitSet, relOptTable, catalogReader, convertedInput, operation,
+ updateColumnList, sourceExpressionList, flattened);
+ default:
+ throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
new file mode 100644
index 0000000..2922725
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentProjectRel;
+
+public class TridentProjectRule extends ConverterRule {
+ public static final TridentProjectRule INSTANCE = new TridentProjectRule();
+
+ private TridentProjectRule() {
+ super(LogicalProject.class, Convention.NONE, TridentLogicalConvention.INSTANCE,
+ "TridentProjectRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final Project project = (Project) rel;
+ final RelNode input = project.getInput();
+
+ return new TridentProjectRel(project.getCluster(),
+ project.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+ convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)), project.getProjects(), project.getRowType());
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
new file mode 100644
index 0000000..abbd680
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.planner.trident.rules;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.schema.Table;
+import org.apache.storm.sql.calcite.ParallelStreamableTable;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentStreamScanRel;
+
+public class TridentScanRule extends ConverterRule {
+ public static final TridentScanRule INSTANCE = new TridentScanRule();
+ public static final int DEFAULT_PARALLELISM_HINT = 1;
+
+ private TridentScanRule() {
+ super(EnumerableTableScan.class, EnumerableConvention.INSTANCE, TridentLogicalConvention.INSTANCE, "TridentScanRule");
+ }
+
+ @Override
+ public RelNode convert(RelNode rel) {
+ final TableScan scan = (TableScan) rel;
+ int parallelismHint = DEFAULT_PARALLELISM_HINT;
+
+ final ParallelStreamableTable parallelTable = scan.getTable().unwrap(ParallelStreamableTable.class);
+ if (parallelTable != null && parallelTable.parallelismHint() != null) {
+ parallelismHint = parallelTable.parallelismHint();
+ }
+
+ final Table table = scan.getTable().unwrap(Table.class);
+ switch (table.getJdbcTableType()) {
+ case STREAM:
+ return new TridentStreamScanRel(scan.getCluster(),
+ scan.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+ scan.getTable(), parallelismHint);
+ default:
+ throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
+ }
+ }
+}
[17/23] storm git commit: STORM-2453 Move non-connectors into the top
directory
Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml
deleted file mode 100644
index 776a54c..0000000
--- a/external/sql/storm-sql-core/pom.xml
+++ /dev/null
@@ -1,279 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <parent>
- <artifactId>storm</artifactId>
- <groupId>org.apache.storm</groupId>
- <version>2.0.0-SNAPSHOT</version>
- <relativePath>../../../pom.xml</relativePath>
- </parent>
-
- <artifactId>storm-sql-core</artifactId>
-
- <developers>
- <developer>
- <id>haohui</id>
- <name>Haohui Mai</name>
- <email>ricetons@gmail.com</email>
- </developer>
- </developers>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-core</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.storm</groupId>
- <artifactId>storm-sql-runtime</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.calcite</groupId>
- <artifactId>calcite-core</artifactId>
- <version>${calcite.version}</version>
- <exclusions>
- <exclusion>
- <groupId>com.fasterxml.jackson.core</groupId>
- <artifactId>jackson-annotations</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>com.fasterxml.jackson.core</groupId>
- <artifactId>jackson-annotations</artifactId>
- <version>${jackson.version}</version>
- </dependency>
- <dependency>
- <groupId>commons-cli</groupId>
- <artifactId>commons-cli</artifactId>
- </dependency>
- <dependency>
- <groupId>commons-lang</groupId>
- <artifactId>commons-lang</artifactId>
- <scope>runtime</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-collections4</artifactId>
- <version>4.1</version>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <sourceDirectory>src/jvm</sourceDirectory>
- <testSourceDirectory>src/test</testSourceDirectory>
- <plugins>
- <plugin>
- <artifactId>maven-resources-plugin</artifactId>
- <executions>
- <execution>
- <id>copy-fmpp-resources</id>
- <phase>initialize</phase>
- <goals>
- <goal>copy-resources</goal>
- </goals>
- <configuration>
- <outputDirectory>${project.build.directory}/codegen</outputDirectory>
- <resources>
- <resource>
- <directory>src/codegen</directory>
- <filtering>false</filtering>
- </resource>
- </resources>
- </configuration>
- </execution>
- <execution>
- <id>copy-java-sources</id>
- <phase>process-sources</phase>
- <goals>
- <goal>copy-resources</goal>
- </goals>
- <configuration>
- <outputDirectory>${basedir}/target/classes/</outputDirectory>
- <resources>
- <resource>
- <directory>src/jvm</directory>
- <filtering>true</filtering>
- </resource>
- <resource>
- <directory>src/test</directory>
- <filtering>true</filtering>
- </resource>
- <resource>
- <directory>target/generated-sources</directory>
- <!-- <include>*/org</include> -->
- <filtering>true</filtering>
- </resource>
- </resources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.2</version>
- <executions>
- <execution>
- <goals>
- <goal>test-jar</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <!-- Extract parser grammar template from calcite-core.jar and put
- it under ${project.build.directory} where all freemarker templates are. -->
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <version>2.8</version>
- <executions>
- <execution>
- <id>unpack-parser-template</id>
- <phase>initialize</phase>
- <goals>
- <goal>unpack</goal>
- </goals>
- <configuration>
- <artifactItems>
- <artifactItem>
- <groupId>org.apache.calcite</groupId>
- <artifactId>calcite-core</artifactId>
- <type>jar</type>
- <overWrite>true</overWrite>
- <outputDirectory>${project.build.directory}/</outputDirectory>
- <includes>**/Parser.jj</includes>
- </artifactItem>
- </artifactItems>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
- as copy and unpack goal are not working together -->
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>appassembler-maven-plugin</artifactId>
- <version>1.9</version>
- <executions>
- <execution>
- <id>create-repo</id>
- <goals>
- <goal>create-repository</goal>
- </goals>
- <configuration>
- <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
- <repositoryLayout>flat</repositoryLayout>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>com.googlecode.fmpp-maven-plugin</groupId>
- <artifactId>fmpp-maven-plugin</artifactId>
- <version>1.0</version>
- <dependencies>
- <dependency>
- <groupId>org.freemarker</groupId>
- <artifactId>freemarker</artifactId>
- <version>2.3.25-incubating</version>
- </dependency>
- </dependencies>
- <executions>
- <execution>
- <id>generate-fmpp-sources</id>
- <phase>generate-sources</phase>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
- <outputDirectory>target/generated-sources</outputDirectory>
- <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>build-helper-maven-plugin</artifactId>
- <version>1.5</version>
- <executions>
- <execution>
- <id>add-generated-sources</id>
- <phase>process-sources</phase>
- <goals>
- <goal>add-source</goal>
- </goals>
- <configuration>
- <sources>
- <source>${project.build.directory}/generated-sources</source>
- </sources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>javacc-maven-plugin</artifactId>
- <version>2.4</version>
- <executions>
- <execution>
- <phase>generate-sources</phase>
- <id>javacc</id>
- <goals>
- <goal>javacc</goal>
- </goals>
- <configuration>
- <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
- <includes>
- <include>**/Parser.jj</include>
- </includes>
- <lookAhead>2</lookAhead>
- <isStatic>false</isStatic>
- <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/config.fmpp b/external/sql/storm-sql-core/src/codegen/config.fmpp
deleted file mode 100644
index be5a792..0000000
--- a/external/sql/storm-sql-core/src/codegen/config.fmpp
+++ /dev/null
@@ -1,23 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http:# www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-data: {
- parser: tdd(../data/Parser.tdd)
-}
-
-freemarkerLinks: {
- includes: includes/
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/data/Parser.tdd b/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
deleted file mode 100644
index b0dccb6..0000000
--- a/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
+++ /dev/null
@@ -1,80 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http:# www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-{
- # Generated parser implementation class package and name
- package: "org.apache.storm.sql.parser.impl",
- class: "StormParserImpl",
-
- # List of import statements.
- imports: [
- "org.apache.calcite.sql.validate.*",
- "org.apache.calcite.util.*",
- "org.apache.storm.sql.parser.*",
- "java.util.*"
- ]
-
- # List of keywords.
- keywords: [
- "LOCATION",
- "INPUTFORMAT",
- "OUTPUTFORMAT",
- "PARALLELISM",
- "STORED",
- "TBLPROPERTIES",
- "JAR"
- ]
-
- # List of methods for parsing custom SQL statements.
- statementParserMethods: [
- "SqlCreateTable()",
- "SqlCreateFunction()"
- ]
-
- # List of methods for parsing custom literals.
- # Example: ParseJsonLiteral().
- literalParserMethods: [
- ]
-
- # List of methods for parsing custom data types.
- dataTypeParserMethods: [
- ]
-
- nonReservedKeywords: [
- ]
-
- createStatementParserMethods: [
- ]
-
- alterStatementParserMethods: [
- ]
-
- dropStatementParserMethods: [
- ]
-
- # List of files in @includes directory that have parser method
- # implementations for custom SQL statements, literals or types
- # given as part of "statementParserMethods", "literalParserMethods" or
- # "dataTypeParserMethods".
- implementationFiles: [
- "parserImpls.ftl"
- ]
-
- includeCompoundIdentifier: true,
- includeBraces: true,
- includeAdditionalDeclarations: false,
- allowBangEqual: false
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/includes/license.ftl
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/includes/license.ftl b/external/sql/storm-sql-core/src/codegen/includes/license.ftl
deleted file mode 100644
index 7e66353..0000000
--- a/external/sql/storm-sql-core/src/codegen/includes/license.ftl
+++ /dev/null
@@ -1,17 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl b/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
deleted file mode 100644
index 4143840..0000000
--- a/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
+++ /dev/null
@@ -1,113 +0,0 @@
-<#-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
- license agreements. See the NOTICE file distributed with this work for additional
- information regarding copyright ownership. The ASF licenses this file to
- You under the Apache License, Version 2.0 (the "License"); you may not use
- this file except in compliance with the License. You may obtain a copy of
- the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
- by applicable law or agreed to in writing, software distributed under the
- License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
- OF ANY KIND, either express or implied. See the License for the specific
- language governing permissions and limitations under the License. -->
-
-
-private void ColumnDef(List<ColumnDefinition> list) :
-{
- SqlParserPos pos;
- SqlIdentifier name;
- SqlDataTypeSpec type;
- ColumnConstraint constraint = null;
- SqlMonotonicity monotonicity = SqlMonotonicity.NOT_MONOTONIC;
-}
-{
- name = SimpleIdentifier() { pos = getPos(); }
- type = DataType()
- [
- <PRIMARY> <KEY>
- [ <ASC> { monotonicity = SqlMonotonicity.INCREASING; }
- | <DESC> { monotonicity = SqlMonotonicity.DECREASING; }
- ]
- { constraint = new ColumnConstraint.PrimaryKey(monotonicity, getPos()); }
- ]
- {
- list.add(new ColumnDefinition(name, type, constraint, pos));
- }
-}
-
-SqlNodeList ColumnDefinitionList() :
-{
- SqlParserPos pos;
- List<ColumnDefinition> list = Lists.newArrayList();
-}
-{
- <LPAREN> { pos = getPos(); }
- ColumnDef(list)
- ( <COMMA> ColumnDef(list) )*
- <RPAREN> {
- return new SqlNodeList(list, pos.plus(getPos()));
- }
-}
-
-/**
- * CREATE EXTERNAL TABLE ( IF NOT EXISTS )?
- * ( database_name '.' )? table_name ( '(' column_def ( ',' column_def )* ')'
- * ( STORED AS INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname )?
- * LOCATION location_uri
- * ( TBLPROPERTIES tbl_properties )?
- * ( AS select_stmt )
- */
-SqlNode SqlCreateTable() :
-{
- SqlParserPos pos;
- SqlIdentifier tblName;
- SqlNodeList fieldList;
- SqlNode location;
- SqlNode parallelism = null;
- SqlNode input_format_class_name = null, output_format_class_name = null;
- SqlNode tbl_properties = null;
- SqlNode select = null;
-}
-{
- <CREATE> { pos = getPos(); }
- <EXTERNAL> <TABLE>
- tblName = CompoundIdentifier()
- fieldList = ColumnDefinitionList()
- [
- <STORED> <AS>
- <INPUTFORMAT> input_format_class_name = StringLiteral()
- <OUTPUTFORMAT> output_format_class_name = StringLiteral()
- ]
- <LOCATION>
- location = StringLiteral()
- [ <PARALLELISM> parallelism = UnsignedNumericLiteral() ]
- [ <TBLPROPERTIES> tbl_properties = StringLiteral() ]
- [ <AS> select = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) ] {
- return new SqlCreateTable(pos, tblName, fieldList,
- input_format_class_name, output_format_class_name, location,
- parallelism, tbl_properties, select);
- }
-}
-
-/**
- * CREATE FUNCTION functionname AS 'classname'
- */
-SqlNode SqlCreateFunction() :
-{
- SqlParserPos pos;
- SqlIdentifier functionName;
- SqlNode className;
- SqlNode jarName = null;
-}
-{
- <CREATE> { pos = getPos(); }
- <FUNCTION>
- functionName = CompoundIdentifier()
- <AS>
- className = StringLiteral()
- [
- <USING> <JAR>
- jarName = StringLiteral()
- ]
- {
- return new SqlCreateFunction(pos, functionName, className, jarName);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
deleted file mode 100644
index 6af71d4..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import org.apache.calcite.DataContext;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-
-import java.util.List;
-import java.util.Map;
-
-public abstract class AbstractTridentProcessor {
- protected Stream outputStream;
- protected DataContext dataContext;
- protected List<CompilingClassLoader> classLoaders;
- /**
- * @return the output stream of the SQL
- */
- public Stream outputStream() {
- return outputStream;
- }
-
- /**
- * Construct the trident topology based on the SQL.
- */
- public abstract TridentTopology build();
-
- /**
- * @return DataContext instance which is used with execution of query
- */
- public DataContext getDataContext() {
- return dataContext;
- }
-
- /**
- * @return Classloaders to compile. They're all chaining so the last classloader can access all classes.
- */
- public List<CompilingClassLoader> getClassLoaders() {
- return classLoaders;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
deleted file mode 100644
index 5dec4af..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.sql.runtime.ChannelHandler;
-
-import java.util.Map;
-
-/**
- * The StormSql class provides standalone, interactive interfaces to execute
- * SQL statements over streaming data.
- * <p>
- * The StormSql class is stateless. The user needs to submit the data
- * definition language (DDL) statements and the query statements in the same
- * batch.
- */
-public abstract class StormSql {
- /**
- * Execute the SQL statements in stand-alone mode. The user can retrieve the result by passing in an instance
- * of {@see ChannelHandler}.
- */
- public abstract void execute(Iterable<String> statements,
- ChannelHandler handler) throws Exception;
-
- /**
- * Submit the SQL statements to Nimbus and run it as a topology.
- */
- public abstract void submit(
- String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
- StormSubmitter.ProgressListener progressListener, String asUser)
- throws Exception;
-
- /**
- * Print out query plan for each query.
- */
- public abstract void explain(Iterable<String> statements) throws Exception;
-
- public static StormSql construct() {
- return new StormSqlImpl();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
deleted file mode 100644
index 007daa7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
+++ /dev/null
@@ -1,290 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.sql.compiler.StormSqlTypeFactoryImpl;
-import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.parser.ColumnConstraint;
-import org.apache.storm.sql.parser.ColumnDefinition;
-import org.apache.storm.sql.parser.SqlCreateFunction;
-import org.apache.storm.sql.parser.SqlCreateTable;
-import org.apache.storm.sql.parser.StormParser;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.TridentTopology;
-
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.jar.Attributes;
-import java.util.jar.JarOutputStream;
-import java.util.jar.Manifest;
-import java.util.zip.ZipEntry;
-
-import static org.apache.storm.sql.compiler.CompilerUtil.TableBuilderInfo;
-
-class StormSqlImpl extends StormSql {
- private final JavaTypeFactory typeFactory = new StormSqlTypeFactoryImpl(
- RelDataTypeSystem.DEFAULT);
- private final SchemaPlus schema = Frameworks.createRootSchema(true);
- private boolean hasUdf = false;
-
- @Override
- public void execute(
- Iterable<String> statements, ChannelHandler result)
- throws Exception {
- Map<String, DataSource> dataSources = new HashMap<>();
- for (String sql : statements) {
- StormParser parser = new StormParser(sql);
- SqlNode node = parser.impl().parseSqlStmtEof();
- if (node instanceof SqlCreateTable) {
- handleCreateTable((SqlCreateTable) node, dataSources);
- } else if (node instanceof SqlCreateFunction) {
- handleCreateFunction((SqlCreateFunction) node);
- } else {
- FrameworkConfig config = buildFrameWorkConfig();
- Planner planner = Frameworks.getPlanner(config);
- SqlNode parse = planner.parse(sql);
- SqlNode validate = planner.validate(parse);
- RelNode tree = planner.convert(validate);
- PlanCompiler compiler = new PlanCompiler(typeFactory);
- AbstractValuesProcessor proc = compiler.compile(tree);
- proc.initialize(dataSources, result);
- }
- }
- }
-
- @Override
- public void submit(
- String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
- StormSubmitter.ProgressListener progressListener, String asUser)
- throws Exception {
- Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
- for (String sql : statements) {
- StormParser parser = new StormParser(sql);
- SqlNode node = parser.impl().parseSqlStmtEof();
- if (node instanceof SqlCreateTable) {
- handleCreateTableForTrident((SqlCreateTable) node, dataSources);
- } else if (node instanceof SqlCreateFunction) {
- handleCreateFunction((SqlCreateFunction) node);
- } else {
- QueryPlanner planner = new QueryPlanner(schema);
- AbstractTridentProcessor processor = planner.compile(dataSources, sql);
- TridentTopology topo = processor.build();
-
- Path jarPath = null;
- try {
- // QueryPlanner on Trident mode configures the topology with compiled classes,
- // so we need to add new classes into topology jar
- // Topology will be serialized and sent to Nimbus, and deserialized and executed in workers.
-
- jarPath = Files.createTempFile("storm-sql", ".jar");
- System.setProperty("storm.jar", jarPath.toString());
- packageTopology(jarPath, processor);
- StormSubmitter.submitTopologyAs(name, stormConf, topo.build(), opts, progressListener, asUser);
- } finally {
- if (jarPath != null) {
- Files.delete(jarPath);
- }
- }
- }
- }
- }
-
- @Override
- public void explain(Iterable<String> statements) throws Exception {
- Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
- for (String sql : statements) {
- StormParser parser = new StormParser(sql);
- SqlNode node = parser.impl().parseSqlStmtEof();
-
- System.out.println("===========================================================");
- System.out.println("query>");
- System.out.println(sql);
- System.out.println("-----------------------------------------------------------");
-
- if (node instanceof SqlCreateTable) {
- handleCreateTableForTrident((SqlCreateTable) node, dataSources);
- System.out.println("No plan presented on DDL");
- } else if (node instanceof SqlCreateFunction) {
- handleCreateFunction((SqlCreateFunction) node);
- System.out.println("No plan presented on DDL");
- } else {
- FrameworkConfig config = buildFrameWorkConfig();
- Planner planner = Frameworks.getPlanner(config);
- SqlNode parse = planner.parse(sql);
- SqlNode validate = planner.validate(parse);
- RelNode tree = planner.convert(validate);
-
- String plan = StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES);
- System.out.println("plan>");
- System.out.println(plan);
- }
-
- System.out.println("===========================================================");
- }
- }
-
- private void packageTopology(Path jar, AbstractTridentProcessor processor) throws IOException {
- Manifest manifest = new Manifest();
- Attributes attr = manifest.getMainAttributes();
- attr.put(Attributes.Name.MANIFEST_VERSION, "1.0");
- attr.put(Attributes.Name.MAIN_CLASS, processor.getClass().getCanonicalName());
- try (JarOutputStream out = new JarOutputStream(
- new BufferedOutputStream(new FileOutputStream(jar.toFile())), manifest)) {
- List<CompilingClassLoader> classLoaders = processor.getClassLoaders();
- if (classLoaders != null && !classLoaders.isEmpty()) {
- for (CompilingClassLoader classLoader : classLoaders) {
- for (Map.Entry<String, ByteArrayOutputStream> e : classLoader.getClasses().entrySet()) {
- out.putNextEntry(new ZipEntry(e.getKey().replace(".", "/") + ".class"));
- out.write(e.getValue().toByteArray());
- out.closeEntry();
- }
- }
- }
- }
- }
-
- private void handleCreateTable(
- SqlCreateTable n, Map<String, DataSource> dataSources) {
- List<FieldInfo> fields = updateSchema(n);
- DataSource ds = DataSourcesRegistry.construct(n.location(), n
- .inputFormatClass(), n.outputFormatClass(), fields);
- if (ds == null) {
- throw new RuntimeException("Cannot construct data source for " + n
- .tableName());
- } else if (dataSources.containsKey(n.tableName())) {
- throw new RuntimeException("Duplicated definition for table " + n
- .tableName());
- }
- dataSources.put(n.tableName(), ds);
- }
-
- private void handleCreateFunction(SqlCreateFunction sqlCreateFunction) throws ClassNotFoundException {
- if(sqlCreateFunction.jarName() != null) {
- throw new UnsupportedOperationException("UDF 'USING JAR' not implemented");
- }
- Method method;
- Function function;
- if ((method=findMethod(sqlCreateFunction.className(), "evaluate")) != null) {
- function = ScalarFunctionImpl.create(method);
- } else if (findMethod(sqlCreateFunction.className(), "add") != null) {
- function = AggregateFunctionImpl.create(Class.forName(sqlCreateFunction.className()));
- } else {
- throw new RuntimeException("Invalid scalar or aggregate function");
- }
- schema.add(sqlCreateFunction.functionName().toUpperCase(), function);
- hasUdf = true;
- }
-
- private Method findMethod(String clazzName, String methodName) throws ClassNotFoundException {
- Class<?> clazz = Class.forName(clazzName);
- for (Method method : clazz.getMethods()) {
- if (method.getName().equals(methodName)) {
- return method;
- }
- }
- return null;
- }
-
- private void handleCreateTableForTrident(
- SqlCreateTable n, Map<String, ISqlTridentDataSource> dataSources) {
- List<FieldInfo> fields = updateSchema(n);
- ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(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);
- }
-
- private List<FieldInfo> updateSchema(SqlCreateTable n) {
- TableBuilderInfo builder = new 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);
- return fields;
- }
-
- private FrameworkConfig buildFrameWorkConfig() {
- if (hasUdf) {
- List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
- sqlOperatorTables.add(SqlStdOperatorTable.instance());
- sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
- false,
- Collections.<String>emptyList(), typeFactory));
- return Frameworks.newConfigBuilder().defaultSchema(schema)
- .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables)).build();
- } else {
- return Frameworks.newConfigBuilder().defaultSchema(schema).build();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
deleted file mode 100644
index 5618647..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.DefaultParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.generated.TopologyInitialStatus;
-import org.apache.storm.utils.Utils;
-
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.Map;
-
-public class StormSqlRunner {
- private static final String OPTION_SQL_FILE_SHORT = "f";
- private static final String OPTION_SQL_FILE_LONG = "file";
- private static final String OPTION_SQL_TOPOLOGY_NAME_SHORT = "t";
- private static final String OPTION_SQL_TOPOLOGY_NAME_LONG = "topology";
- private static final String OPTION_SQL_EXPLAIN_SHORT = "e";
- private static final String OPTION_SQL_EXPLAIN_LONG = "explain";
-
- public static void main(String[] args) throws Exception {
- Options options = buildOptions();
- CommandLineParser parser = new DefaultParser();
- CommandLine commandLine = parser.parse(options, args);
-
- if (!commandLine.hasOption(OPTION_SQL_FILE_LONG)) {
- printUsageAndExit(options, OPTION_SQL_FILE_LONG + " is required");
- }
-
- String filePath = commandLine.getOptionValue(OPTION_SQL_FILE_LONG);
- List<String> stmts = Files.readAllLines(Paths.get(filePath), StandardCharsets.UTF_8);
- StormSql sql = StormSql.construct();
- @SuppressWarnings("unchecked")
- Map<String, ?> conf = Utils.readStormConfig();
-
- if (commandLine.hasOption(OPTION_SQL_EXPLAIN_LONG)) {
- sql.explain(stmts);
- } else if (commandLine.hasOption(OPTION_SQL_TOPOLOGY_NAME_LONG)) {
- String topoName = commandLine.getOptionValue(OPTION_SQL_TOPOLOGY_NAME_LONG);
- SubmitOptions submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
- sql.submit(topoName, stmts, conf, submitOptions, null, null);
- } else {
- printUsageAndExit(options, "Either " + OPTION_SQL_TOPOLOGY_NAME_LONG + " or " + OPTION_SQL_EXPLAIN_LONG +
- " must be presented");
- }
- }
-
- private static void printUsageAndExit(Options options, String message) {
- System.out.println(message);
- HelpFormatter formatter = new HelpFormatter();
- formatter.printHelp("storm-sql-runner ", options);
- System.exit(1);
- }
-
- private static Options buildOptions() {
- Options options = new Options();
- options.addOption(OPTION_SQL_FILE_SHORT, OPTION_SQL_FILE_LONG, true, "REQUIRED SQL file which has sql statements");
- options.addOption(OPTION_SQL_TOPOLOGY_NAME_SHORT, OPTION_SQL_TOPOLOGY_NAME_LONG, true, "Topology name to submit");
- options.addOption(OPTION_SQL_EXPLAIN_SHORT, OPTION_SQL_EXPLAIN_LONG, false, "Activate explain mode (topology name will be ignored)");
- return options;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
deleted file mode 100644
index c6b584d..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.storm.sql.calcite;
-
-import org.apache.calcite.rel.stream.Delta;
-import org.apache.calcite.schema.StreamableTable;
-
-/**
- * Table that can be converted to a stream. This table also has its parallelism information.
- *
- * @see Delta
- */
-public interface ParallelStreamableTable extends StreamableTable {
-
- /**
- * Returns parallelism hint of this table. Returns null if don't know.
- */
- Integer parallelismHint();
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
deleted file mode 100644
index 2e237c0..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.*;
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlMonotonicity;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
-import org.apache.storm.sql.calcite.ParallelStreamableTable;
-import org.apache.storm.sql.parser.ColumnConstraint;
-
-import java.util.ArrayList;
-
-import static org.apache.calcite.rel.RelFieldCollation.Direction;
-import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
-import static org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
-import static org.apache.calcite.rel.RelFieldCollation.NullDirection;
-import static org.apache.calcite.sql.validate.SqlMonotonicity.INCREASING;
-
-public class CompilerUtil {
- public static String escapeJavaString(String s, boolean nullMeansNull) {
- if(s == null) {
- return nullMeansNull ? "null" : "\"\"";
- } else {
- String s1 = Util.replace(s, "\\", "\\\\");
- String s2 = Util.replace(s1, "\"", "\\\"");
- String s3 = Util.replace(s2, "\n\r", "\\n");
- String s4 = Util.replace(s3, "\n", "\\n");
- String s5 = Util.replace(s4, "\r", "\\r");
- return "\"" + s5 + "\"";
- }
- }
-
- public static class TableBuilderInfo {
- private final RelDataTypeFactory typeFactory;
-
- public TableBuilderInfo(RelDataTypeFactory typeFactory) {
- this.typeFactory = typeFactory;
- }
-
- private static class FieldType {
- private final String name;
- private final RelDataType relDataType;
-
- private FieldType(String name, RelDataType relDataType) {
- this.name = name;
- this.relDataType = relDataType;
- }
-
- }
-
- private final ArrayList<FieldType> fields = new ArrayList<>();
- private final ArrayList<Object[]> rows = new ArrayList<>();
- private int primaryKey = -1;
- private Integer parallelismHint;
- private SqlMonotonicity primaryKeyMonotonicity;
- private Statistic stats;
-
- public TableBuilderInfo field(String name, SqlTypeName type) {
- return field(name, typeFactory.createSqlType(type));
- }
-
- public TableBuilderInfo field(String name, RelDataType type) {
- fields.add(new FieldType(name, type));
- return this;
- }
-
- public TableBuilderInfo field(String name, SqlDataTypeSpec type, ColumnConstraint constraint) {
- RelDataType dataType = type.deriveType(typeFactory);
- if (constraint instanceof ColumnConstraint.PrimaryKey) {
- ColumnConstraint.PrimaryKey pk = (ColumnConstraint.PrimaryKey) constraint;
- Preconditions.checkState(primaryKey == -1, "There are more than one primary key in the table");
- primaryKey = fields.size();
- primaryKeyMonotonicity = pk.monotonicity();
- }
- fields.add(new FieldType(name, dataType));
- return this;
- }
-
- public TableBuilderInfo statistics(Statistic stats) {
- this.stats = stats;
- return this;
- }
-
- @VisibleForTesting
- public TableBuilderInfo rows(Object[] data) {
- rows.add(data);
- return this;
- }
-
- public TableBuilderInfo parallelismHint(int parallelismHint) {
- this.parallelismHint = parallelismHint;
- return this;
- }
-
- public StreamableTable build() {
- final Statistic stat = buildStatistic();
- final Table tbl = new Table() {
- @Override
- public RelDataType getRowType(
- RelDataTypeFactory relDataTypeFactory) {
- RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
- for (FieldType f : fields) {
- b.add(f.name, f.relDataType);
- }
- return b.build();
- }
-
- @Override
- public Statistic getStatistic() {
- return stat != null ? stat : Statistics.of(rows.size(),
- ImmutableList.<ImmutableBitSet>of());
- }
-
- @Override
- public Schema.TableType getJdbcTableType() {
- return Schema.TableType.STREAM;
- }
- };
-
- return new ParallelStreamableTable() {
- @Override
- public Integer parallelismHint() {
- return parallelismHint;
- }
-
- @Override
- public Table stream() {
- return tbl;
- }
-
- @Override
- public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
- return tbl.getRowType(relDataTypeFactory);
- }
-
- @Override
- public Statistic getStatistic() {
- return tbl.getStatistic();
- }
-
- @Override
- public Schema.TableType getJdbcTableType() {
- return Schema.TableType.STREAM;
- }
- };
- }
-
- private Statistic buildStatistic() {
- if (stats != null || primaryKey == -1) {
- return stats;
- }
- Direction dir = primaryKeyMonotonicity == INCREASING ? ASCENDING : DESCENDING;
- RelFieldCollation collation = new RelFieldCollation(primaryKey, dir, NullDirection.UNSPECIFIED);
- return Statistics.of(fields.size(), ImmutableList.of(ImmutableBitSet.of(primaryKey)),
- ImmutableList.of(RelCollations.of(collation)));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
deleted file mode 100644
index 5ac95e0..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.calcite.adapter.enumerable.JavaRowFormat;
-import org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.linq4j.function.Function1;
-import org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.calcite.linq4j.tree.BlockStatement;
-import org.apache.calcite.linq4j.tree.ClassDeclaration;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.linq4j.tree.MemberDeclaration;
-import org.apache.calcite.linq4j.tree.ParameterExpression;
-import org.apache.calcite.linq4j.tree.Types;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexProgramBuilder;
-import org.apache.calcite.util.BuiltInMethod;
-import org.apache.calcite.util.Pair;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.Type;
-import java.util.List;
-
-/**
- * Compiles a scalar expression ({@link org.apache.calcite.rex.RexNode}) to Java source code String.
- *
- * This code is inspired by JaninoRexCompiler in Calcite, but while it is returning {@link org.apache.calcite.interpreter.Scalar} which is executable,
- * we need to pass the source code to compile and serialize instance so that it can be executed on worker efficiently.
- */
-public class RexNodeToJavaCodeCompiler {
- private final RexBuilder rexBuilder;
-
- public RexNodeToJavaCodeCompiler(RexBuilder rexBuilder) {
- this.rexBuilder = rexBuilder;
- }
-
- public BlockStatement compileToBlock(List<RexNode> nodes, RelDataType inputRowType) {
- final RexProgramBuilder programBuilder =
- new RexProgramBuilder(inputRowType, rexBuilder);
- for (RexNode node : nodes) {
- programBuilder.addProject(node, null);
- }
-
- return compileToBlock(programBuilder.getProgram());
- }
-
- public BlockStatement compileToBlock(final RexProgram program) {
- final ParameterExpression context_ =
- Expressions.parameter(Context.class, "context");
- final ParameterExpression outputValues_ =
- Expressions.parameter(Object[].class, "outputValues");
-
- return compileToBlock(program, context_, outputValues_).toBlock();
- }
-
- public String compile(List<RexNode> nodes, RelDataType inputRowType, String className) {
- final RexProgramBuilder programBuilder =
- new RexProgramBuilder(inputRowType, rexBuilder);
- for (RexNode node : nodes) {
- programBuilder.addProject(node, null);
- }
-
- return compile(programBuilder.getProgram(), className);
- }
-
- public String compile(final RexProgram program, String className) {
- final ParameterExpression context_ =
- Expressions.parameter(Context.class, "context");
- final ParameterExpression outputValues_ =
- Expressions.parameter(Object[].class, "outputValues");
-
- BlockBuilder builder = compileToBlock(program, context_, outputValues_);
- return baz(context_, outputValues_, builder.toBlock(), className);
- }
-
- private BlockBuilder compileToBlock(final RexProgram program, ParameterExpression context_,
- ParameterExpression outputValues_) {
- RelDataType inputRowType = program.getInputRowType();
- final BlockBuilder builder = new BlockBuilder();
- final JavaTypeFactoryImpl javaTypeFactory =
- new JavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
-
- final RexToLixTranslator.InputGetter inputGetter =
- new RexToLixTranslator.InputGetterImpl(
- ImmutableList.of(
- Pair.<Expression, PhysType>of(
- Expressions.field(context_,
- BuiltInMethod.CONTEXT_VALUES.field),
- PhysTypeImpl.of(javaTypeFactory, inputRowType,
- JavaRowFormat.ARRAY, false))));
- final Function1<String, RexToLixTranslator.InputGetter> correlates =
- new Function1<String, RexToLixTranslator.InputGetter>() {
- public RexToLixTranslator.InputGetter apply(String a0) {
- throw new UnsupportedOperationException();
- }
- };
- final Expression root =
- Expressions.field(context_, BuiltInMethod.CONTEXT_ROOT.field);
- final List<Expression> list =
- RexToLixTranslator.translateProjects(program, javaTypeFactory, builder,
- null, root, inputGetter, correlates);
- for (int i = 0; i < list.size(); i++) {
- builder.add(
- Expressions.statement(
- Expressions.assign(
- Expressions.arrayIndex(outputValues_,
- Expressions.constant(i)),
- list.get(i))));
- }
-
- return builder;
- }
-
- /** Given a method that implements {@link ExecutableExpression#execute(Context, Object[])},
- * adds a bridge method that implements {@link ExecutableExpression#execute(Context)}, and
- * compiles. */
- static String baz(ParameterExpression context_,
- ParameterExpression outputValues_, BlockStatement block, String className) {
- final List<MemberDeclaration> declarations = Lists.newArrayList();
-
- // public void execute(Context, Object[] outputValues)
- declarations.add(
- Expressions.methodDecl(Modifier.PUBLIC, void.class,
- StormBuiltInMethod.EXPR_EXECUTE2.method.getName(),
- ImmutableList.of(context_, outputValues_), block));
-
- // public Object execute(Context)
- final BlockBuilder builder = new BlockBuilder();
- final Expression values_ = builder.append("values",
- Expressions.newArrayBounds(Object.class, 1,
- Expressions.constant(1)));
- builder.add(
- Expressions.statement(
- Expressions.call(
- Expressions.parameter(ExecutableExpression.class, "this"),
- StormBuiltInMethod.EXPR_EXECUTE2.method, context_, values_)));
- builder.add(
- Expressions.return_(null,
- Expressions.arrayIndex(values_, Expressions.constant(0))));
- declarations.add(
- Expressions.methodDecl(Modifier.PUBLIC, Object.class,
- StormBuiltInMethod.EXPR_EXECUTE1.method.getName(),
- ImmutableList.of(context_), builder.toBlock()));
-
- final ClassDeclaration classDeclaration =
- Expressions.classDecl(Modifier.PUBLIC, className, null,
- ImmutableList.<Type>of(ExecutableExpression.class), declarations);
-
- return Expressions.toString(Lists.newArrayList(classDeclaration), "\n", false);
- }
-
- enum StormBuiltInMethod {
- EXPR_EXECUTE1(ExecutableExpression.class, "execute", Context.class),
- EXPR_EXECUTE2(ExecutableExpression.class, "execute", Context.class, Object[].class);
-
- public final Method method;
- public final Constructor constructor;
- public final Field field;
-
- public static final ImmutableMap<Method, BuiltInMethod> MAP;
-
- static {
- final ImmutableMap.Builder<Method, BuiltInMethod> builder =
- ImmutableMap.builder();
- for (BuiltInMethod value : BuiltInMethod.values()) {
- if (value.method != null) {
- builder.put(value.method, value);
- }
- }
- MAP = builder.build();
- }
-
- private StormBuiltInMethod(Method method, Constructor constructor, Field field) {
- this.method = method;
- this.constructor = constructor;
- this.field = field;
- }
-
- /**
- * Defines a method.
- */
- StormBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
- this(Types.lookupMethod(clazz, methodName, argumentTypes), null, null);
- }
-
- /**
- * Defines a constructor.
- */
- StormBuiltInMethod(Class clazz, Class... argumentTypes) {
- this(null, Types.lookupConstructor(clazz, argumentTypes), null);
- }
-
- /**
- * Defines a field.
- */
- StormBuiltInMethod(Class clazz, String fieldName, boolean dummy) {
- this(null, null, Types.lookupField(clazz, fieldName));
- assert dummy : "dummy value for method overloading must be true";
- }
- }
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
deleted file mode 100644
index 21ca063..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-public class StormSqlTypeFactoryImpl extends JavaTypeFactoryImpl {
-
- public StormSqlTypeFactoryImpl() {
- }
-
- public StormSqlTypeFactoryImpl(RelDataTypeSystem typeSystem) {
- super(typeSystem);
- }
-
- @Override
- public RelDataType toSql(RelDataType type) {
- if (type instanceof JavaType) {
- JavaType javaType = (JavaType) type;
- SqlTypeName sqlTypeName = JavaToSqlTypeConversionRules.instance().lookup(javaType.getJavaClass());
- if (sqlTypeName == null) {
- sqlTypeName = SqlTypeName.ANY;
- }
- return createTypeWithNullability(createSqlType(sqlTypeName), type.isNullable());
- }
- return super.toSql(type);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
deleted file mode 100644
index 9dc4ba8..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.storm.tuple.Values;
-
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Built-in implementations for some of the standard aggregation operations.
- * Aggregations can be implemented as a class with the following methods viz. init, add and result.
- * The class could contain only static methods, only non-static methods or be generic.
- */
-public class BuiltinAggregateFunctions {
- // binds the type information and the class implementing the aggregation
- public static class TypeClass {
- public static class GenericType {
- }
-
- public final Type ty;
- public final Class<?> clazz;
-
- private TypeClass(Type ty, Class<?> clazz) {
- this.ty = ty;
- this.clazz = clazz;
- }
-
- static TypeClass of(Type ty, Class<?> clazz) {
- return new TypeClass(ty, clazz);
- }
- }
-
- static final Map<String, List<TypeClass>> TABLE = new HashMap<>();
-
- public static class ByteSum {
- public static Byte init() {
- return 0;
- }
-
- public static Byte add(Byte accumulator, Byte val) {
- return (byte) (accumulator + val);
- }
-
- public static Byte result(Byte accumulator) {
- return accumulator;
- }
- }
-
- public static class ShortSum {
- public static Short init() {
- return 0;
- }
-
- public static Short add(Short accumulator, Short val) {
- return (short) (accumulator + val);
- }
-
- public static Short result(Short accumulator) {
- return accumulator;
- }
- }
-
- public static class IntSum {
- public static Integer init() {
- return 0;
- }
-
- public static Integer add(Integer accumulator, Integer val) {
- return accumulator + val;
- }
-
- public static Integer result(Integer accumulator) {
- return accumulator;
- }
- }
-
- public static class LongSum {
- public static Long init() {
- return 0L;
- }
-
- public static Long add(Long accumulator, Long val) {
- return accumulator + val;
- }
-
- public static Long result(Long accumulator) {
- return accumulator;
- }
- }
-
- public static class FloatSum {
- public static Float init() {
- return 0.0f;
- }
-
- public static Float add(Float accumulator, Float val) {
- return accumulator + val;
- }
-
- public static Float result(Float accumulator) {
- return accumulator;
- }
- }
-
- public static class DoubleSum {
- public static Double init() {
- return 0.0;
- }
-
- public static Double add(Double accumulator, Double val) {
- return accumulator + val;
- }
-
- public static Double result(Double accumulator) {
- return accumulator;
- }
- }
-
- public static class Max<T extends Comparable<T>> {
- public T init() {
- return null;
- }
-
- public T add(T accumulator, T val) {
- return (accumulator == null || accumulator.compareTo(val) < 0) ? val : accumulator;
- }
-
- public T result(T accumulator) {
- return accumulator;
- }
- }
-
- public static class Min<T extends Comparable<T>> {
- public T init() {
- return null;
- }
-
- public T add(T accumulator, T val) {
- return (accumulator == null || accumulator.compareTo(val) > 0) ? val : accumulator;
- }
-
- public T result(T accumulator) {
- return accumulator;
- }
- }
-
- public static class IntAvg {
- private int count;
-
- public Integer init() {
- return 0;
- }
-
- public Integer add(Integer accumulator, Integer val) {
- ++count;
- return accumulator + val;
- }
-
- public Integer result(Integer accumulator) {
- Integer result = accumulator / count;
- count = 0;
- return result;
- }
- }
-
- public static class DoubleAvg {
- private int count;
-
- public Double init() {
- return 0.0;
- }
-
- public Double add(Double accumulator, Double val) {
- ++count;
- return accumulator + val;
- }
-
- public Double result(Double accumulator) {
- Double result = accumulator / count;
- count = 0;
- return result;
- }
- }
-
- public static class Count {
- public static Long init() {
- return 0L;
- }
-
- public static Long add(Long accumulator, Values vals) {
- for (Object val : vals) {
- if (val == null) {
- return accumulator;
- }
- }
- return accumulator + 1;
- }
-
- public static Long result(Long accumulator) {
- return accumulator;
- }
- }
-
- static {
- TABLE.put("SUM", ImmutableList.of(
- TypeClass.of(float.class, FloatSum.class),
- TypeClass.of(double.class, DoubleSum.class),
- TypeClass.of(byte.class, ByteSum.class),
- TypeClass.of(short.class, ShortSum.class),
- TypeClass.of(long.class, LongSum.class),
- TypeClass.of(int.class, IntSum.class)));
- TABLE.put("AVG", ImmutableList.of(
- TypeClass.of(double.class, DoubleAvg.class),
- TypeClass.of(int.class, IntAvg.class)));
- TABLE.put("COUNT", ImmutableList.of(TypeClass.of(long.class, Count.class)));
- TABLE.put("MAX", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Max.class)));
- TABLE.put("MIN", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Min.class)));
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
deleted file mode 100644
index 01546ed..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.storm.sql.compiler.CompilerUtil;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.HashSet;
-import java.util.Set;
-
-public class PlanCompiler {
- private static final Logger LOG = LoggerFactory.getLogger(PlanCompiler.class);
-
- private static final Joiner NEW_LINE_JOINER = Joiner.on("\n");
- private static final String PACKAGE_NAME = "org.apache.storm.sql.generated";
- private static final String PROLOGUE = NEW_LINE_JOINER.join(
- "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "",
- "import java.util.Iterator;", "import java.util.Map;", "import java.util.HashMap;",
- "import java.util.List;", "import java.util.ArrayList;",
- "import java.util.LinkedHashMap;",
- "import org.apache.storm.tuple.Values;",
- "import org.apache.storm.sql.runtime.AbstractChannelHandler;",
- "import org.apache.storm.sql.runtime.Channels;",
- "import org.apache.storm.sql.runtime.ChannelContext;",
- "import org.apache.storm.sql.runtime.ChannelHandler;",
- "import org.apache.storm.sql.runtime.DataSource;",
- "import org.apache.storm.sql.runtime.AbstractValuesProcessor;",
- "import com.google.common.collect.ArrayListMultimap;",
- "import com.google.common.collect.Multimap;",
- "import org.apache.calcite.interpreter.Context;",
- "import org.apache.calcite.interpreter.StormContext;",
- "import org.apache.calcite.DataContext;",
- "import org.apache.storm.sql.runtime.calcite.StormDataContext;",
- "public final class Processor extends AbstractValuesProcessor {",
- " public final static DataContext dataContext = new StormDataContext();",
- "");
- private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
- " @Override",
- " public void initialize(Map<String, DataSource> data,",
- " ChannelHandler result) {",
- " ChannelContext r = Channels.chain(Channels.voidContext(), result);",
- ""
- );
-
- private final JavaTypeFactory typeFactory;
-
- public PlanCompiler(JavaTypeFactory typeFactory) {
- this.typeFactory = typeFactory;
- }
-
- private String generateJavaSource(RelNode root) throws Exception {
- StringWriter sw = new StringWriter();
- try (PrintWriter pw = new PrintWriter(sw)) {
- RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
- printPrologue(pw);
- compiler.traverse(root);
- printMain(pw, root);
- printEpilogue(pw);
- }
- return sw.toString();
- }
-
- private void printMain(PrintWriter pw, RelNode root) {
- Set<TableScan> tables = new HashSet<>();
- pw.print(INITIALIZER_PROLOGUE);
- chainOperators(pw, root, tables);
- for (TableScan n : tables) {
- String escaped = CompilerUtil.escapeJavaString(
- Joiner.on('.').join(n.getTable().getQualifiedName()), true);
- String r = NEW_LINE_JOINER.join(
- " if (!data.containsKey(%1$s))",
- " throw new RuntimeException(\"Cannot find table \" + %1$s);",
- " data.get(%1$s).open(CTX_%2$d);",
- "");
- pw.print(String.format(r, escaped, n.getId()));
- }
- pw.print(" }\n");
- }
-
- private void chainOperators(PrintWriter pw, RelNode root, Set<TableScan> tables) {
- doChainOperators(pw, root, tables, "r");
- }
-
- private void doChainOperators(PrintWriter pw, RelNode node, Set<TableScan> tables, String parentCtx) {
- pw.print(
- String.format(" ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
- node.getId(), parentCtx, RelNodeCompiler.getStageName(node)));
- String currentCtx = String.format("CTX_%d", node.getId());
- if (node instanceof TableScan) {
- tables.add((TableScan) node);
- }
- for (RelNode i : node.getInputs()) {
- doChainOperators(pw, i, tables, currentCtx);
- }
- }
-
- public AbstractValuesProcessor compile(RelNode plan) throws Exception {
- String javaCode = generateJavaSource(plan);
- LOG.debug("Compiling... source code {}", javaCode);
- ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
- PACKAGE_NAME + ".Processor",
- javaCode, null);
- return (AbstractValuesProcessor) cl.loadClass(
- PACKAGE_NAME + ".Processor").newInstance();
- }
-
- private static void printEpilogue(
- PrintWriter pw) throws Exception {
- pw.print("}\n");
- }
-
- private static void printPrologue(PrintWriter pw) {
- pw.append(PROLOGUE);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
deleted file mode 100644
index afed8a9..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.compiler.backends.standalone;
-
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.*;
-import org.apache.calcite.rel.stream.Delta;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public abstract class PostOrderRelNodeVisitor<T> {
- public final T traverse(RelNode n) throws Exception {
- List<T> inputStreams = new ArrayList<>();
- for (RelNode input : n.getInputs()) {
- inputStreams.add(traverse(input));
- }
-
- if (n instanceof Aggregate) {
- return visitAggregate((Aggregate) n, inputStreams);
- } else if (n instanceof Calc) {
- return visitCalc((Calc) n, inputStreams);
- } else if (n instanceof Collect) {
- return visitCollect((Collect) n, inputStreams);
- } else if (n instanceof Correlate) {
- return visitCorrelate((Correlate) n, inputStreams);
- } else if (n instanceof Delta) {
- return visitDelta((Delta) n, inputStreams);
- } else if (n instanceof Exchange) {
- return visitExchange((Exchange) n, inputStreams);
- } else if (n instanceof Project) {
- return visitProject((Project) n, inputStreams);
- } else if (n instanceof Filter) {
- return visitFilter((Filter) n, inputStreams);
- } else if (n instanceof Sample) {
- return visitSample((Sample) n, inputStreams);
- } else if (n instanceof Sort) {
- return visitSort((Sort) n, inputStreams);
- } else if (n instanceof TableModify) {
- return visitTableModify((TableModify) n, inputStreams);
- } else if (n instanceof TableScan) {
- return visitTableScan((TableScan) n, inputStreams);
- } else if (n instanceof Uncollect) {
- return visitUncollect((Uncollect) n, inputStreams);
- } else if (n instanceof Window) {
- return visitWindow((Window) n, inputStreams);
- } else if (n instanceof Join) {
- return visitJoin((Join) n, inputStreams);
- } else {
- return defaultValue(n, inputStreams);
- }
- }
-
- public T visitAggregate(Aggregate aggregate, List<T> inputStreams) throws Exception {
- return defaultValue(aggregate, inputStreams);
- }
-
- public T visitCalc(Calc calc, List<T> inputStreams) throws Exception {
- return defaultValue(calc, inputStreams);
- }
-
- public T visitCollect(Collect collect, List<T> inputStreams) throws Exception {
- return defaultValue(collect, inputStreams);
- }
-
- public T visitCorrelate(Correlate correlate, List<T> inputStreams) throws Exception {
- return defaultValue(correlate, inputStreams);
- }
-
- public T visitDelta(Delta delta, List<T> inputStreams) throws Exception {
- return defaultValue(delta, inputStreams);
- }
-
- public T visitExchange(Exchange exchange, List<T> inputStreams) throws Exception {
- return defaultValue(exchange, inputStreams);
- }
-
- public T visitProject(Project project, List<T> inputStreams) throws Exception {
- return defaultValue(project, inputStreams);
- }
-
- public T visitFilter(Filter filter, List<T> inputStreams) throws Exception {
- return defaultValue(filter, inputStreams);
- }
-
- public T visitSample(Sample sample, List<T> inputStreams) throws Exception {
- return defaultValue(sample, inputStreams);
- }
-
- public T visitSort(Sort sort, List<T> inputStreams) throws Exception {
- return defaultValue(sort, inputStreams);
- }
-
- public T visitTableModify(TableModify modify, List<T> inputStreams) throws Exception {
- return defaultValue(modify, inputStreams);
- }
-
- public T visitTableScan(TableScan scan, List<T> inputStreams) throws Exception {
- return defaultValue(scan, inputStreams);
- }
-
- public T visitUncollect(Uncollect uncollect, List<T> inputStreams) throws Exception {
- return defaultValue(uncollect, inputStreams);
- }
-
- public T visitWindow(Window window, List<T> inputStreams) throws Exception {
- return defaultValue(window, inputStreams);
- }
-
- public T visitJoin(Join join, List<T> inputStreams) throws Exception {
- return defaultValue(join, inputStreams);
- }
-
- public T defaultValue(RelNode n, List<T> inputStreams) {
- return null;
- }
-}