You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ro...@apache.org on 2017/08/04 17:09:44 UTC
[1/7] beam git commit: move BeamRecord to sdk/core
Repository: beam
Updated Branches:
refs/heads/DSL_SQL 10962a34d -> 8f922f74b
move BeamRecord to sdk/core
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/52933a64
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/52933a64
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/52933a64
Branch: refs/heads/DSL_SQL
Commit: 52933a640393a107eddbd3d88670507b03595e1f
Parents: 10962a3
Author: mingmxu <mi...@ebay.com>
Authored: Wed Aug 2 01:20:50 2017 -0700
Committer: mingmxu <mi...@ebay.com>
Committed: Wed Aug 2 23:52:29 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 84 ++++++
.../org/apache/beam/sdk/values/BeamRecord.java | 279 ++++++++++++++++++
.../beam/sdk/values/BeamRecordTypeProvider.java | 59 ++++
.../apache/beam/sdk/extensions/sql/BeamSql.java | 2 +-
.../beam/sdk/extensions/sql/BeamSqlCli.java | 4 +-
.../sdk/extensions/sql/schema/BeamSqlRow.java | 293 +------------------
.../extensions/sql/schema/BeamSqlRowCoder.java | 79 ++---
.../extensions/sql/schema/BeamSqlRowType.java | 91 +++++-
8 files changed, 555 insertions(+), 336 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
new file mode 100644
index 0000000..ad27f4e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.coders;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.BeamRecordTypeProvider;
+
+/**
+ * A {@link Coder} for {@link BeamRecord}. It wraps the {@link Coder} for each element directly.
+ */
+@Experimental
+public class BeamRecordCoder extends CustomCoder<BeamRecord> {
+ private static final ListCoder<Integer> nullListCoder = ListCoder.of(BigEndianIntegerCoder.of());
+ private static final InstantCoder instantCoder = InstantCoder.of();
+
+ private BeamRecordTypeProvider recordType;
+ private List<Coder> coderArray;
+
+ public BeamRecordCoder(BeamRecordTypeProvider recordType, List<Coder> coderArray) {
+ this.recordType = recordType;
+ this.coderArray = coderArray;
+ }
+
+ @Override
+ public void encode(BeamRecord value, OutputStream outStream)
+ throws CoderException, IOException {
+ nullListCoder.encode(value.getNullFields(), outStream);
+ for (int idx = 0; idx < value.size(); ++idx) {
+ if (value.getNullFields().contains(idx)) {
+ continue;
+ }
+
+ coderArray.get(idx).encode(value.getInteger(idx), outStream);
+ }
+
+ instantCoder.encode(value.getWindowStart(), outStream);
+ instantCoder.encode(value.getWindowEnd(), outStream);
+ }
+
+ @Override
+ public BeamRecord decode(InputStream inStream) throws CoderException, IOException {
+ List<Integer> nullFields = nullListCoder.decode(inStream);
+
+ BeamRecord record = new BeamRecord(recordType);
+ record.setNullFields(nullFields);
+ for (int idx = 0; idx < recordType.size(); ++idx) {
+ if (nullFields.contains(idx)) {
+ continue;
+ }
+
+ record.addField(idx, coderArray.get(idx).decode(inStream));
+ }
+
+ record.setWindowStart(instantCoder.decode(inStream));
+ record.setWindowEnd(instantCoder.decode(inStream));
+
+ return record;
+ }
+
+ @Override
+ public void verifyDeterministic()
+ throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
new file mode 100644
index 0000000..d1c1c17
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.values;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.joda.time.Instant;
+
+/**
+ * {@link org.apache.beam.sdk.values.BeamRecord}, self-described with
+ * {@link BeamRecordTypeProvider}, represents one element in a
+ * {@link org.apache.beam.sdk.values.PCollection}.
+ */
+@Experimental
+public class BeamRecord implements Serializable {
+ //null values are indexed here, to handle properly in Coder.
+ private List<Integer> nullFields = new ArrayList<>();
+ private List<Object> dataValues;
+ private BeamRecordTypeProvider dataType;
+
+ private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
+ private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
+
+ public BeamRecord(BeamRecordTypeProvider dataType) {
+ this.dataType = dataType;
+ this.dataValues = new ArrayList<>();
+ for (int idx = 0; idx < dataType.size(); ++idx) {
+ dataValues.add(null);
+ nullFields.add(idx);
+ }
+ }
+
+ public BeamRecord(BeamRecordTypeProvider dataType, List<Object> dataValues) {
+ this(dataType);
+ for (int idx = 0; idx < dataValues.size(); ++idx) {
+ addField(idx, dataValues.get(idx));
+ }
+ }
+
+ public void updateWindowRange(BeamRecord upstreamRecord, BoundedWindow window){
+ windowStart = upstreamRecord.windowStart;
+ windowEnd = upstreamRecord.windowEnd;
+
+ if (window instanceof IntervalWindow) {
+ IntervalWindow iWindow = (IntervalWindow) window;
+ windowStart = iWindow.start();
+ windowEnd = iWindow.end();
+ }
+ }
+
+ public void addField(String fieldName, Object fieldValue) {
+ addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
+ }
+
+ public void addField(int index, Object fieldValue) {
+ if (fieldValue == null) {
+ return;
+ } else {
+ if (nullFields.contains(index)) {
+ nullFields.remove(nullFields.indexOf(index));
+ }
+ }
+
+ dataType.validateValueType(index, fieldValue);
+ dataValues.set(index, fieldValue);
+ }
+
+ public Object getFieldValue(String fieldName) {
+ return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
+ }
+
+ public byte getByte(String fieldName) {
+ return (Byte) getFieldValue(fieldName);
+ }
+
+ public short getShort(String fieldName) {
+ return (Short) getFieldValue(fieldName);
+ }
+
+ public int getInteger(String fieldName) {
+ return (Integer) getFieldValue(fieldName);
+ }
+
+ public float getFloat(String fieldName) {
+ return (Float) getFieldValue(fieldName);
+ }
+
+ public double getDouble(String fieldName) {
+ return (Double) getFieldValue(fieldName);
+ }
+
+ public long getLong(String fieldName) {
+ return (Long) getFieldValue(fieldName);
+ }
+
+ public String getString(String fieldName) {
+ return (String) getFieldValue(fieldName);
+ }
+
+ public Date getDate(String fieldName) {
+ return (Date) getFieldValue(fieldName);
+ }
+
+ public GregorianCalendar getGregorianCalendar(String fieldName) {
+ return (GregorianCalendar) getFieldValue(fieldName);
+ }
+
+ public BigDecimal getBigDecimal(String fieldName) {
+ return (BigDecimal) getFieldValue(fieldName);
+ }
+
+ public boolean getBoolean(String fieldName) {
+ return (boolean) getFieldValue(fieldName);
+ }
+
+ public Object getFieldValue(int fieldIdx) {
+ if (nullFields.contains(fieldIdx)) {
+ return null;
+ }
+
+ return dataValues.get(fieldIdx);
+ }
+
+ public byte getByte(int idx) {
+ return (Byte) getFieldValue(idx);
+ }
+
+ public short getShort(int idx) {
+ return (Short) getFieldValue(idx);
+ }
+
+ public int getInteger(int idx) {
+ return (Integer) getFieldValue(idx);
+ }
+
+ public float getFloat(int idx) {
+ return (Float) getFieldValue(idx);
+ }
+
+ public double getDouble(int idx) {
+ return (Double) getFieldValue(idx);
+ }
+
+ public long getLong(int idx) {
+ return (Long) getFieldValue(idx);
+ }
+
+ public String getString(int idx) {
+ return (String) getFieldValue(idx);
+ }
+
+ public Date getDate(int idx) {
+ return (Date) getFieldValue(idx);
+ }
+
+ public GregorianCalendar getGregorianCalendar(int idx) {
+ return (GregorianCalendar) getFieldValue(idx);
+ }
+
+ public BigDecimal getBigDecimal(int idx) {
+ return (BigDecimal) getFieldValue(idx);
+ }
+
+ public boolean getBoolean(int idx) {
+ return (boolean) getFieldValue(idx);
+ }
+
+ public int size() {
+ return dataValues.size();
+ }
+
+ public List<Object> getDataValues() {
+ return dataValues;
+ }
+
+ public void setDataValues(List<Object> dataValues) {
+ this.dataValues = dataValues;
+ }
+
+ public BeamRecordTypeProvider getDataType() {
+ return dataType;
+ }
+
+ public void setDataType(BeamRecordTypeProvider dataType) {
+ this.dataType = dataType;
+ }
+
+ public void setNullFields(List<Integer> nullFields) {
+ this.nullFields = nullFields;
+ }
+
+ public List<Integer> getNullFields() {
+ return nullFields;
+ }
+
+ /**
+ * is the specified field NULL?
+ */
+ public boolean isNull(int idx) {
+ return nullFields.contains(idx);
+ }
+
+ public Instant getWindowStart() {
+ return windowStart;
+ }
+
+ public Instant getWindowEnd() {
+ return windowEnd;
+ }
+
+ public void setWindowStart(Instant windowStart) {
+ this.windowStart = windowStart;
+ }
+
+ public void setWindowEnd(Instant windowEnd) {
+ this.windowEnd = windowEnd;
+ }
+
+ @Override
+ public String toString() {
+ return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
+ + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
+ }
+
+ /**
+ * Return data fields as key=value.
+ */
+ public String valueInString() {
+ StringBuilder sb = new StringBuilder();
+ for (int idx = 0; idx < size(); ++idx) {
+ sb.append(
+ String.format(",%s=%s", getDataType().getFieldsName().get(idx), getFieldValue(idx)));
+ }
+ return sb.substring(1);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ BeamRecord other = (BeamRecord) obj;
+ return toString().equals(other.toString());
+ }
+
+ @Override public int hashCode() {
+ return 31 * (31 * getDataType().hashCode() + getDataValues().hashCode())
+ + getNullFields().hashCode();
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java
new file mode 100644
index 0000000..63a961c
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.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.beam.sdk.values;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+
+/**
+ * The default type provider used in {@link BeamRecord}.
+ */
+@Experimental
+public class BeamRecordTypeProvider implements Serializable{
+ private List<String> fieldsName;
+
+ public BeamRecordTypeProvider(List<String> fieldsName) {
+ this.fieldsName = fieldsName;
+ }
+
+ /**
+ * Validate input fieldValue for a field.
+ * @throws IllegalArgumentException throw exception when the validation fails.
+ */
+ public void validateValueType(int index, Object fieldValue)
+ throws IllegalArgumentException{
+ //do nothing by default.
+ }
+
+ public List<String> getFieldsName(){
+ return fieldsName;
+ }
+
+ public String getFieldByIndex(int index){
+ return fieldsName.get(index);
+ }
+
+ public int findIndexOfField(String fieldName){
+ return fieldsName.indexOf(fieldName);
+ }
+
+ public int size(){
+ return fieldsName.size();
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
index e0d7a78..0dabf40 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
@@ -167,7 +167,7 @@ public class BeamSql {
BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
getSqlEnv().registerTable(sourceTag.getId(),
- new BeamPCollectionTable(sourceStream, sourceCoder.getTableSchema()));
+ new BeamPCollectionTable(sourceStream, sourceCoder.getSqlRecordType()));
}
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
index 3bea46a..967dee5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -56,8 +56,8 @@ public class BeamSqlCli {
/**
* compile SQL, and return a {@link Pipeline}.
*/
- public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline
- , BeamSqlEnv sqlEnv) throws Exception{
+ public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline,
+ BeamSqlEnv sqlEnv) throws Exception{
PCollection<BeamSqlRow> resultStream =
sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
return resultStream;
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
index 2e0efe8..cb5c7ea 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
@@ -17,298 +17,25 @@
*/
package org.apache.beam.sdk.extensions.sql.schema;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.joda.time.Instant;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.PCollection;
/**
- * Represent a generic ROW record in Beam SQL.
- *
+ * {@link BeamSqlRow} represents one row element in a {@link PCollection},
+ * with type provider {@link BeamSqlRowType}.
*/
-public class BeamSqlRow implements Serializable {
- private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
- static {
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
- }
-
- private List<Integer> nullFields = new ArrayList<>();
- private List<Object> dataValues;
- private BeamSqlRowType dataType;
-
- private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
- private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
-
- public BeamSqlRow(BeamSqlRowType dataType) {
- this.dataType = dataType;
- this.dataValues = new ArrayList<>();
- for (int idx = 0; idx < dataType.size(); ++idx) {
- dataValues.add(null);
- nullFields.add(idx);
- }
- }
-
+public class BeamSqlRow extends BeamRecord {
public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
- this(dataType);
- for (int idx = 0; idx < dataValues.size(); ++idx) {
- addField(idx, dataValues.get(idx));
- }
- }
-
- public void updateWindowRange(BeamSqlRow upstreamRecord, BoundedWindow window){
- windowStart = upstreamRecord.windowStart;
- windowEnd = upstreamRecord.windowEnd;
-
- if (window instanceof IntervalWindow) {
- IntervalWindow iWindow = (IntervalWindow) window;
- windowStart = iWindow.start();
- windowEnd = iWindow.end();
- }
- }
-
- public void addField(String fieldName, Object fieldValue) {
- addField(dataType.getFieldsName().indexOf(fieldName), fieldValue);
- }
-
- public void addField(int index, Object fieldValue) {
- if (fieldValue == null) {
- return;
- } else {
- if (nullFields.contains(index)) {
- nullFields.remove(nullFields.indexOf(index));
- }
- }
-
- validateValueType(index, fieldValue);
- dataValues.set(index, fieldValue);
- }
-
- private void validateValueType(int index, Object fieldValue) {
- SqlTypeName fieldType = CalciteUtils.getFieldType(dataType, index);
- Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(CalciteUtils.toJavaType(fieldType));
- if (javaClazz == null) {
- throw new UnsupportedOperationException("Data type: " + fieldType + " not supported yet!");
- }
-
- if (!fieldValue.getClass().equals(javaClazz)) {
- throw new IllegalArgumentException(
- String.format("[%s](%s) doesn't match type [%s]",
- fieldValue, fieldValue.getClass(), fieldType)
- );
- }
- }
-
- public Object getFieldValue(String fieldName) {
- return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
- }
-
- public byte getByte(String fieldName) {
- return (Byte) getFieldValue(fieldName);
- }
-
- public short getShort(String fieldName) {
- return (Short) getFieldValue(fieldName);
- }
-
- public int getInteger(String fieldName) {
- return (Integer) getFieldValue(fieldName);
- }
-
- public float getFloat(String fieldName) {
- return (Float) getFieldValue(fieldName);
- }
-
- public double getDouble(String fieldName) {
- return (Double) getFieldValue(fieldName);
- }
-
- public long getLong(String fieldName) {
- return (Long) getFieldValue(fieldName);
- }
-
- public String getString(String fieldName) {
- return (String) getFieldValue(fieldName);
- }
-
- public Date getDate(String fieldName) {
- return (Date) getFieldValue(fieldName);
- }
-
- public GregorianCalendar getGregorianCalendar(String fieldName) {
- return (GregorianCalendar) getFieldValue(fieldName);
- }
-
- public BigDecimal getBigDecimal(String fieldName) {
- return (BigDecimal) getFieldValue(fieldName);
- }
-
- public boolean getBoolean(String fieldName) {
- return (boolean) getFieldValue(fieldName);
- }
-
- public Object getFieldValue(int fieldIdx) {
- if (nullFields.contains(fieldIdx)) {
- return null;
- }
-
- return dataValues.get(fieldIdx);
- }
-
- public byte getByte(int idx) {
- return (Byte) getFieldValue(idx);
+ super(dataType, dataValues);
}
- public short getShort(int idx) {
- return (Short) getFieldValue(idx);
- }
-
- public int getInteger(int idx) {
- return (Integer) getFieldValue(idx);
- }
-
- public float getFloat(int idx) {
- return (Float) getFieldValue(idx);
- }
-
- public double getDouble(int idx) {
- return (Double) getFieldValue(idx);
- }
-
- public long getLong(int idx) {
- return (Long) getFieldValue(idx);
- }
-
- public String getString(int idx) {
- return (String) getFieldValue(idx);
- }
-
- public Date getDate(int idx) {
- return (Date) getFieldValue(idx);
- }
-
- public GregorianCalendar getGregorianCalendar(int idx) {
- return (GregorianCalendar) getFieldValue(idx);
- }
-
- public BigDecimal getBigDecimal(int idx) {
- return (BigDecimal) getFieldValue(idx);
- }
-
- public boolean getBoolean(int idx) {
- return (boolean) getFieldValue(idx);
- }
-
- public int size() {
- return dataValues.size();
- }
-
- public List<Object> getDataValues() {
- return dataValues;
- }
-
- public void setDataValues(List<Object> dataValues) {
- this.dataValues = dataValues;
- }
-
- public BeamSqlRowType getDataType() {
- return dataType;
- }
-
- public void setDataType(BeamSqlRowType dataType) {
- this.dataType = dataType;
- }
-
- public void setNullFields(List<Integer> nullFields) {
- this.nullFields = nullFields;
- }
-
- public List<Integer> getNullFields() {
- return nullFields;
- }
-
- /**
- * is the specified field NULL?
- */
- public boolean isNull(int idx) {
- return nullFields.contains(idx);
- }
-
- public Instant getWindowStart() {
- return windowStart;
- }
-
- public Instant getWindowEnd() {
- return windowEnd;
- }
-
- public void setWindowStart(Instant windowStart) {
- this.windowStart = windowStart;
- }
-
- public void setWindowEnd(Instant windowEnd) {
- this.windowEnd = windowEnd;
- }
-
- @Override
- public String toString() {
- return "BeamSqlRow [nullFields=" + nullFields + ", dataValues=" + dataValues + ", dataType="
- + dataType + ", windowStart=" + windowStart + ", windowEnd=" + windowEnd + "]";
- }
-
- /**
- * Return data fields as key=value.
- */
- public String valueInString() {
- StringBuilder sb = new StringBuilder();
- for (int idx = 0; idx < size(); ++idx) {
- sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx)));
- }
- return sb.substring(1);
+ public BeamSqlRow(BeamSqlRowType dataType) {
+ super(dataType);
}
@Override
- public boolean equals(Object obj) {
- if (this == obj) {
- return true;
- }
- if (obj == null) {
- return false;
- }
- if (getClass() != obj.getClass()) {
- return false;
- }
- BeamSqlRow other = (BeamSqlRow) obj;
- return toString().equals(other.toString());
- }
-
- @Override public int hashCode() {
- return 31 * (31 * dataType.hashCode() + dataValues.hashCode()) + nullFields.hashCode();
+ public BeamSqlRowType getDataType() {
+ return (BeamSqlRowType) super.getDataType();
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
index bf097d4..3d760c4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.extensions.sql.schema;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.sql.Types;
import java.util.Date;
import java.util.GregorianCalendar;
import java.util.List;
@@ -34,13 +35,12 @@ import org.apache.beam.sdk.coders.DoubleCoder;
import org.apache.beam.sdk.coders.InstantCoder;
import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
/**
* A {@link Coder} encodes {@link BeamSqlRow}.
*/
public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
- private BeamSqlRowType tableSchema;
+ private BeamSqlRowType sqlRecordType;
private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
@@ -52,58 +52,59 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
private static final ByteCoder byteCoder = ByteCoder.of();
- public BeamSqlRowCoder(BeamSqlRowType tableSchema) {
- this.tableSchema = tableSchema;
+ public BeamSqlRowCoder(BeamSqlRowType sqlRecordType) {
+ this.sqlRecordType = sqlRecordType;
}
@Override
- public void encode(BeamSqlRow value, OutputStream outStream) throws CoderException, IOException {
+ public void encode(BeamSqlRow value, OutputStream outStream)
+ throws CoderException, IOException {
listCoder.encode(value.getNullFields(), outStream);
for (int idx = 0; idx < value.size(); ++idx) {
if (value.getNullFields().contains(idx)) {
continue;
}
- switch (CalciteUtils.getFieldType(value.getDataType(), idx)) {
- case INTEGER:
+ switch (sqlRecordType.getFieldsType().get(idx)) {
+ case Types.INTEGER:
intCoder.encode(value.getInteger(idx), outStream);
break;
- case SMALLINT:
+ case Types.SMALLINT:
intCoder.encode((int) value.getShort(idx), outStream);
break;
- case TINYINT:
+ case Types.TINYINT:
byteCoder.encode(value.getByte(idx), outStream);
break;
- case DOUBLE:
+ case Types.DOUBLE:
doubleCoder.encode(value.getDouble(idx), outStream);
break;
- case FLOAT:
+ case Types.FLOAT:
doubleCoder.encode((double) value.getFloat(idx), outStream);
break;
- case DECIMAL:
+ case Types.DECIMAL:
bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
break;
- case BIGINT:
+ case Types.BIGINT:
longCoder.encode(value.getLong(idx), outStream);
break;
- case VARCHAR:
- case CHAR:
+ case Types.VARCHAR:
+ case Types.CHAR:
stringCoder.encode(value.getString(idx), outStream);
break;
- case TIME:
+ case Types.TIME:
longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
break;
- case DATE:
- case TIMESTAMP:
+ case Types.DATE:
+ case Types.TIMESTAMP:
longCoder.encode(value.getDate(idx).getTime(), outStream);
break;
- case BOOLEAN:
+ case Types.BOOLEAN:
byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
break;
default:
throw new UnsupportedOperationException(
- "Data type: " + value.getDataType().getFieldsType().get(idx) + " not supported yet!");
+ "Data type: " + sqlRecordType.getFieldsType().get(idx) + " not supported yet!");
}
}
@@ -115,55 +116,55 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
List<Integer> nullFields = listCoder.decode(inStream);
- BeamSqlRow record = new BeamSqlRow(tableSchema);
+ BeamSqlRow record = new BeamSqlRow(sqlRecordType);
record.setNullFields(nullFields);
- for (int idx = 0; idx < tableSchema.size(); ++idx) {
+ for (int idx = 0; idx < sqlRecordType.size(); ++idx) {
if (nullFields.contains(idx)) {
continue;
}
- switch (CalciteUtils.getFieldType(tableSchema, idx)) {
- case INTEGER:
+ switch (sqlRecordType.getFieldsType().get(idx)) {
+ case Types.INTEGER:
record.addField(idx, intCoder.decode(inStream));
break;
- case SMALLINT:
+ case Types.SMALLINT:
record.addField(idx, intCoder.decode(inStream).shortValue());
break;
- case TINYINT:
+ case Types.TINYINT:
record.addField(idx, byteCoder.decode(inStream));
break;
- case DOUBLE:
+ case Types.DOUBLE:
record.addField(idx, doubleCoder.decode(inStream));
break;
- case FLOAT:
+ case Types.FLOAT:
record.addField(idx, doubleCoder.decode(inStream).floatValue());
break;
- case BIGINT:
+ case Types.BIGINT:
record.addField(idx, longCoder.decode(inStream));
break;
- case DECIMAL:
+ case Types.DECIMAL:
record.addField(idx, bigDecimalCoder.decode(inStream));
break;
- case VARCHAR:
- case CHAR:
+ case Types.VARCHAR:
+ case Types.CHAR:
record.addField(idx, stringCoder.decode(inStream));
break;
- case TIME:
+ case Types.TIME:
GregorianCalendar calendar = new GregorianCalendar();
calendar.setTime(new Date(longCoder.decode(inStream)));
record.addField(idx, calendar);
break;
- case DATE:
- case TIMESTAMP:
+ case Types.DATE:
+ case Types.TIMESTAMP:
record.addField(idx, new Date(longCoder.decode(inStream)));
break;
- case BOOLEAN:
+ case Types.BOOLEAN:
record.addField(idx, byteCoder.decode(inStream) == 1);
break;
default:
throw new UnsupportedOperationException("Data type: "
- + CalciteUtils.toCalciteType(tableSchema.getFieldsType().get(idx))
+ + sqlRecordType.getFieldsType().get(idx)
+ " not supported yet!");
}
}
@@ -174,8 +175,8 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
return record;
}
- public BeamSqlRowType getTableSchema() {
- return tableSchema;
+ public BeamSqlRowType getSqlRecordType() {
+ return sqlRecordType;
}
@Override
http://git-wip-us.apache.org/repos/asf/beam/blob/52933a64/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
index 018fe81..7584dad 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
@@ -17,24 +17,93 @@
*/
package org.apache.beam.sdk.extensions.sql.schema;
-import com.google.auto.value.AutoValue;
-import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.values.BeamRecordTypeProvider;
/**
- * Field type information in {@link BeamSqlRow}.
+ * Type provider for {@link BeamSqlRow} with SQL types.
+ *
+ * <p>Limited SQL types are supported now, visit
+ * <a href="https://beam.apache.org/blog/2017/07/21/sql-dsl.html#data-type">data types</a>
+ * for more details.
*
*/
-@AutoValue
-public abstract class BeamSqlRowType implements Serializable {
- public abstract List<String> getFieldsName();
- public abstract List<Integer> getFieldsType();
+public class BeamSqlRowType extends BeamRecordTypeProvider {
+ private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
+ static {
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
+ }
+
+ public List<Integer> fieldsType;
+
+ protected BeamSqlRowType(List<String> fieldsName) {
+ super(fieldsName);
+ }
+
+ public BeamSqlRowType(List<String> fieldsName, List<Integer> fieldsType) {
+ super(fieldsName);
+ this.fieldsType = fieldsType;
+ }
+
+ public static BeamSqlRowType create(List<String> fieldNames,
+ List<Integer> fieldTypes) {
+ return new BeamSqlRowType(fieldNames, fieldTypes);
+ }
+
+ @Override
+ public void validateValueType(int index, Object fieldValue) throws IllegalArgumentException {
+ int fieldType = fieldsType.get(index);
+ Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(fieldType);
+ if (javaClazz == null) {
+ throw new IllegalArgumentException("Data type: " + fieldType + " not supported yet!");
+ }
+
+ if (!fieldValue.getClass().equals(javaClazz)) {
+ throw new IllegalArgumentException(
+ String.format("[%s](%s) doesn't match type [%s]",
+ fieldValue, fieldValue.getClass(), fieldType)
+ );
+ }
+ }
+
+ public List<Integer> getFieldsType() {
+ return fieldsType;
+ }
- public static BeamSqlRowType create(List<String> fieldNames, List<Integer> fieldTypes) {
- return new AutoValue_BeamSqlRowType(fieldNames, fieldTypes);
+ @Override
+ public boolean equals(Object obj) {
+ if (obj != null && obj instanceof BeamSqlRowType) {
+ BeamSqlRowType ins = (BeamSqlRowType) obj;
+ return fieldsType.equals(ins.getFieldsType()) && getFieldsName().equals(ins.getFieldsName());
+ } else {
+ return false;
+ }
}
- public int size() {
- return getFieldsName().size();
+ @Override
+ public int hashCode() {
+ return 31 * getFieldsName().hashCode() + getFieldsType().hashCode();
}
}
[7/7] beam git commit: Closes #3675
Posted by ro...@apache.org.
Closes #3675
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/8f922f74
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/8f922f74
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/8f922f74
Branch: refs/heads/DSL_SQL
Commit: 8f922f74b693795f25d7c2d4a9da9ea3de5a92eb
Parents: 10962a3 89109b8
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Fri Aug 4 10:08:38 2017 -0700
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Aug 4 10:08:38 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 98 ++++++
.../org/apache/beam/sdk/values/BeamRecord.java | 267 ++++++++++++++++
.../apache/beam/sdk/values/BeamRecordType.java | 70 +++++
.../apache/beam/sdk/extensions/sql/BeamSql.java | 22 +-
.../beam/sdk/extensions/sql/BeamSqlCli.java | 10 +-
.../beam/sdk/extensions/sql/BeamSqlEnv.java | 6 +-
.../extensions/sql/example/BeamSqlExample.java | 27 +-
.../interpreter/BeamSqlExpressionExecutor.java | 6 +-
.../sql/impl/interpreter/BeamSqlFnExecutor.java | 6 +-
.../operator/BeamSqlCaseExpression.java | 4 +-
.../operator/BeamSqlCastExpression.java | 4 +-
.../interpreter/operator/BeamSqlExpression.java | 8 +-
.../operator/BeamSqlInputRefExpression.java | 4 +-
.../interpreter/operator/BeamSqlPrimitive.java | 6 +-
.../operator/BeamSqlReinterpretExpression.java | 4 +-
.../operator/BeamSqlUdfExpression.java | 4 +-
.../operator/BeamSqlWindowEndExpression.java | 4 +-
.../operator/BeamSqlWindowExpression.java | 4 +-
.../operator/BeamSqlWindowStartExpression.java | 4 +-
.../arithmetic/BeamSqlArithmeticExpression.java | 4 +-
.../comparison/BeamSqlCompareExpression.java | 4 +-
.../comparison/BeamSqlIsNotNullExpression.java | 4 +-
.../comparison/BeamSqlIsNullExpression.java | 4 +-
.../date/BeamSqlCurrentDateExpression.java | 4 +-
.../date/BeamSqlCurrentTimeExpression.java | 4 +-
.../date/BeamSqlCurrentTimestampExpression.java | 4 +-
.../date/BeamSqlDateCeilExpression.java | 4 +-
.../date/BeamSqlDateFloorExpression.java | 4 +-
.../operator/date/BeamSqlExtractExpression.java | 4 +-
.../operator/logical/BeamSqlAndExpression.java | 4 +-
.../operator/logical/BeamSqlNotExpression.java | 4 +-
.../operator/logical/BeamSqlOrExpression.java | 4 +-
.../math/BeamSqlMathBinaryExpression.java | 4 +-
.../math/BeamSqlMathUnaryExpression.java | 4 +-
.../operator/math/BeamSqlPiExpression.java | 4 +-
.../operator/math/BeamSqlRandExpression.java | 4 +-
.../math/BeamSqlRandIntegerExpression.java | 4 +-
.../string/BeamSqlCharLengthExpression.java | 4 +-
.../string/BeamSqlConcatExpression.java | 4 +-
.../string/BeamSqlInitCapExpression.java | 4 +-
.../operator/string/BeamSqlLowerExpression.java | 4 +-
.../string/BeamSqlOverlayExpression.java | 4 +-
.../string/BeamSqlPositionExpression.java | 4 +-
.../string/BeamSqlSubstringExpression.java | 4 +-
.../operator/string/BeamSqlTrimExpression.java | 4 +-
.../operator/string/BeamSqlUpperExpression.java | 4 +-
.../sql/impl/planner/BeamQueryPlanner.java | 4 +-
.../sql/impl/rel/BeamAggregationRel.java | 38 +--
.../extensions/sql/impl/rel/BeamFilterRel.java | 11 +-
.../extensions/sql/impl/rel/BeamIOSinkRel.java | 6 +-
.../sql/impl/rel/BeamIOSourceRel.java | 13 +-
.../sql/impl/rel/BeamIntersectRel.java | 4 +-
.../extensions/sql/impl/rel/BeamJoinRel.java | 75 +++--
.../extensions/sql/impl/rel/BeamMinusRel.java | 4 +-
.../extensions/sql/impl/rel/BeamProjectRel.java | 11 +-
.../extensions/sql/impl/rel/BeamRelNode.java | 4 +-
.../sql/impl/rel/BeamSetOperatorRelBase.java | 18 +-
.../extensions/sql/impl/rel/BeamSortRel.java | 49 ++-
.../extensions/sql/impl/rel/BeamUnionRel.java | 4 +-
.../extensions/sql/impl/rel/BeamValuesRel.java | 15 +-
.../transform/BeamAggregationTransforms.java | 47 +--
.../sql/impl/transform/BeamJoinTransforms.java | 65 ++--
.../transform/BeamSetOperatorsTransforms.java | 24 +-
.../sql/impl/transform/BeamSqlFilterFn.java | 6 +-
.../transform/BeamSqlOutputToConsoleFn.java | 4 +-
.../sql/impl/transform/BeamSqlProjectFn.java | 14 +-
.../extensions/sql/impl/utils/CalciteUtils.java | 10 +-
.../extensions/sql/schema/BaseBeamTable.java | 6 +-
.../sql/schema/BeamPCollectionTable.java | 13 +-
.../sql/schema/BeamSqlRecordHelper.java | 217 +++++++++++++
.../sql/schema/BeamSqlRecordType.java | 168 ++++++++++
.../sdk/extensions/sql/schema/BeamSqlRow.java | 314 -------------------
.../extensions/sql/schema/BeamSqlRowCoder.java | 185 -----------
.../extensions/sql/schema/BeamSqlRowType.java | 40 ---
.../sdk/extensions/sql/schema/BeamSqlTable.java | 7 +-
.../extensions/sql/schema/BeamTableUtils.java | 14 +-
.../sql/schema/kafka/BeamKafkaCSVTable.java | 38 +--
.../sql/schema/kafka/BeamKafkaTable.java | 20 +-
.../sql/schema/text/BeamTextCSVTable.java | 12 +-
.../schema/text/BeamTextCSVTableIOReader.java | 14 +-
.../schema/text/BeamTextCSVTableIOWriter.java | 16 +-
.../sql/schema/text/BeamTextTable.java | 4 +-
.../sql/BeamSqlDslAggregationTest.java | 80 ++---
.../beam/sdk/extensions/sql/BeamSqlDslBase.java | 51 ++-
.../extensions/sql/BeamSqlDslFilterTest.java | 26 +-
.../sdk/extensions/sql/BeamSqlDslJoinTest.java | 26 +-
.../extensions/sql/BeamSqlDslProjectTest.java | 64 ++--
.../extensions/sql/BeamSqlDslUdfUdafTest.java | 24 +-
.../beam/sdk/extensions/sql/TestUtils.java | 30 +-
.../interpreter/BeamSqlFnExecutorTestBase.java | 10 +-
.../sql/impl/rel/BeamIntersectRelTest.java | 6 +-
.../rel/BeamJoinRelBoundedVsBoundedTest.java | 10 +-
.../rel/BeamJoinRelUnboundedVsBoundedTest.java | 10 +-
.../BeamJoinRelUnboundedVsUnboundedTest.java | 10 +-
.../sql/impl/rel/BeamMinusRelTest.java | 6 +-
.../impl/rel/BeamSetOperatorRelBaseTest.java | 4 +-
.../sql/impl/rel/BeamSortRelTest.java | 12 +-
.../sql/impl/rel/BeamUnionRelTest.java | 6 +-
.../sql/impl/rel/BeamValuesRelTest.java | 8 +-
.../sdk/extensions/sql/impl/rel/CheckSize.java | 8 +-
...mSqlBuiltinFunctionsIntegrationTestBase.java | 17 +-
...amSqlComparisonOperatorsIntegrationTest.java | 11 +-
.../BeamSqlDateFunctionsIntegrationTest.java | 12 +-
.../extensions/sql/mock/MockedBoundedTable.java | 24 +-
.../sdk/extensions/sql/mock/MockedTable.java | 8 +-
.../sql/mock/MockedUnboundedTable.java | 18 +-
.../sql/schema/BeamSqlRowCoderTest.java | 8 +-
.../sql/schema/kafka/BeamKafkaCSVTableTest.java | 14 +-
.../sql/schema/text/BeamTextCSVTableTest.java | 16 +-
.../transform/BeamAggregationTransformTest.java | 72 ++---
.../schema/transform/BeamTransformBaseTest.java | 18 +-
111 files changed, 1519 insertions(+), 1250 deletions(-)
----------------------------------------------------------------------
[3/7] beam git commit: refactor BeamRecord, BeamRecordType,
BeamSqlRecordType, BeamRecordCoder
Posted by ro...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
index 46cab09..e3c6aec 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
@@ -20,11 +20,11 @@ package org.apache.beam.sdk.extensions.sql;
import java.sql.Types;
import java.util.Arrays;
import java.util.Iterator;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -39,24 +39,24 @@ public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
*/
@Test
public void testUdaf() throws Exception {
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "squaresum"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int2", "squaresum"),
Arrays.asList(Types.INTEGER, Types.INTEGER));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("f_int2", 0);
record.addField("squaresum", 30);
String sql1 = "SELECT f_int2, squaresum1(f_int) AS `squaresum`"
+ " FROM PCOLLECTION GROUP BY f_int2";
- PCollection<BeamSqlRow> result1 =
+ PCollection<BeamRecord> result1 =
boundedInput1.apply("testUdaf1",
BeamSql.simpleQuery(sql1).withUdaf("squaresum1", SquareSum.class));
PAssert.that(result1).containsInAnyOrder(record);
String sql2 = "SELECT f_int2, squaresum2(f_int) AS `squaresum`"
+ " FROM PCOLLECTION GROUP BY f_int2";
- PCollection<BeamSqlRow> result2 =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+ PCollection<BeamRecord> result2 =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("PCOLLECTION"), boundedInput1)
.apply("testUdaf2",
BeamSql.query(sql2).withUdaf("squaresum2", SquareSum.class));
PAssert.that(result2).containsInAnyOrder(record);
@@ -69,22 +69,22 @@ public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
*/
@Test
public void testUdf() throws Exception{
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "cubicvalue"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int", "cubicvalue"),
Arrays.asList(Types.INTEGER, Types.INTEGER));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("f_int", 2);
record.addField("cubicvalue", 8);
String sql1 = "SELECT f_int, cubic1(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
- PCollection<BeamSqlRow> result1 =
+ PCollection<BeamRecord> result1 =
boundedInput1.apply("testUdf1",
BeamSql.simpleQuery(sql1).withUdf("cubic1", CubicInteger.class));
PAssert.that(result1).containsInAnyOrder(record);
String sql2 = "SELECT f_int, cubic2(f_int) as cubicvalue FROM PCOLLECTION WHERE f_int = 2";
- PCollection<BeamSqlRow> result2 =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("PCOLLECTION"), boundedInput1)
+ PCollection<BeamRecord> result2 =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("PCOLLECTION"), boundedInput1)
.apply("testUdf2",
BeamSql.query(sql2).withUdf("cubic2", CubicInteger.class));
PAssert.that(result2).containsInAnyOrder(record);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
index 9995b0a..63b6ca8 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
@@ -21,9 +21,9 @@ package org.apache.beam.sdk.extensions.sql;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.BeamRecord;
/**
* Test utilities.
@@ -32,7 +32,7 @@ public class TestUtils {
/**
* A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}.
*/
- public static class BeamSqlRow2StringDoFn extends DoFn<BeamSqlRow, String> {
+ public static class BeamSqlRow2StringDoFn extends DoFn<BeamRecord, String> {
@ProcessElement
public void processElement(ProcessContext ctx) {
ctx.output(ctx.element().valueInString());
@@ -42,9 +42,9 @@ public class TestUtils {
/**
* Convert list of {@code BeamSqlRow} to list of {@code String}.
*/
- public static List<String> beamSqlRows2Strings(List<BeamSqlRow> rows) {
+ public static List<String> beamSqlRows2Strings(List<BeamRecord> rows) {
List<String> strs = new ArrayList<>();
- for (BeamSqlRow row : rows) {
+ for (BeamRecord row : rows) {
strs.add(row.valueInString());
}
@@ -69,8 +69,8 @@ public class TestUtils {
* {@code}
*/
public static class RowsBuilder {
- private BeamSqlRowType type;
- private List<BeamSqlRow> rows = new ArrayList<>();
+ private BeamSqlRecordType type;
+ private List<BeamRecord> rows = new ArrayList<>();
/**
* Create a RowsBuilder with the specified row type info.
@@ -86,7 +86,7 @@ public class TestUtils {
* @args pairs of column type and column names.
*/
public static RowsBuilder of(final Object... args) {
- BeamSqlRowType beamSQLRowType = buildBeamSqlRowType(args);
+ BeamSqlRecordType beamSQLRowType = buildBeamSqlRowType(args);
RowsBuilder builder = new RowsBuilder();
builder.type = beamSQLRowType;
@@ -103,7 +103,7 @@ public class TestUtils {
* )}</pre>
* @beamSQLRowType the record type.
*/
- public static RowsBuilder of(final BeamSqlRowType beamSQLRowType) {
+ public static RowsBuilder of(final BeamSqlRecordType beamSQLRowType) {
RowsBuilder builder = new RowsBuilder();
builder.type = beamSQLRowType;
@@ -130,7 +130,7 @@ public class TestUtils {
return this;
}
- public List<BeamSqlRow> getRows() {
+ public List<BeamRecord> getRows() {
return rows;
}
@@ -153,7 +153,7 @@ public class TestUtils {
* )
* }</pre>
*/
- public static BeamSqlRowType buildBeamSqlRowType(Object... args) {
+ public static BeamSqlRecordType buildBeamSqlRowType(Object... args) {
List<Integer> types = new ArrayList<>();
List<String> names = new ArrayList<>();
@@ -162,7 +162,7 @@ public class TestUtils {
names.add((String) args[i + 1]);
}
- return BeamSqlRowType.create(names, types);
+ return BeamSqlRecordType.create(names, types);
}
/**
@@ -179,12 +179,12 @@ public class TestUtils {
* )
* }</pre>
*/
- public static List<BeamSqlRow> buildRows(BeamSqlRowType type, List args) {
- List<BeamSqlRow> rows = new ArrayList<>();
+ public static List<BeamRecord> buildRows(BeamSqlRecordType type, List args) {
+ List<BeamRecord> rows = new ArrayList<>();
int fieldCount = type.size();
for (int i = 0; i < args.size(); i += fieldCount) {
- BeamSqlRow row = new BeamSqlRow(type);
+ BeamRecord row = new BeamRecord(type);
for (int j = 0; j < fieldCount; j++) {
row.addField(j, args.get(i + j));
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
index 388c556..4da7790 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutorTestBase.java
@@ -24,8 +24,8 @@ import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelDataTypeSystem;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.adapter.java.JavaTypeFactory;
import org.apache.calcite.config.Lex;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
@@ -57,8 +57,8 @@ public class BeamSqlFnExecutorTestBase {
RelDataTypeSystem.DEFAULT);
public static RelDataType relDataType;
- public static BeamSqlRowType beamRowType;
- public static BeamSqlRow record;
+ public static BeamSqlRecordType beamRowType;
+ public static BeamRecord record;
public static RelBuilder relBuilder;
@@ -71,7 +71,7 @@ public class BeamSqlFnExecutorTestBase {
.add("order_time", SqlTypeName.BIGINT).build();
beamRowType = CalciteUtils.toBeamRowType(relDataType);
- record = new BeamSqlRow(beamRowType);
+ record = new BeamRecord(beamRowType);
record.addField(0, 1234567L);
record.addField(1, 0);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
index 5a3f65d..a51cc30 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
@@ -23,9 +23,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -77,7 +77,7 @@ public class BeamIntersectRelTest {
+ "SELECT order_id, site_id, price "
+ "FROM ORDER_DETAILS2 ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -100,7 +100,7 @@ public class BeamIntersectRelTest {
+ "SELECT order_id, site_id, price "
+ "FROM ORDER_DETAILS2 ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).satisfies(new CheckSize(3));
PAssert.that(rows).containsInAnyOrder(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
index c4f6350..dde1540 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelBoundedVsBoundedTest.java
@@ -23,9 +23,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -77,7 +77,7 @@ public class BeamJoinRelBoundedVsBoundedTest {
+ " o1.order_id=o2.site_id AND o2.price=o1.site_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.INTEGER, "order_id",
@@ -102,7 +102,7 @@ public class BeamJoinRelBoundedVsBoundedTest {
+ " o1.order_id=o2.site_id AND o2.price=o1.site_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
pipeline.enableAbandonedNodeEnforcement(false);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -130,7 +130,7 @@ public class BeamJoinRelBoundedVsBoundedTest {
+ " o1.order_id=o2.site_id AND o2.price=o1.site_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.INTEGER, "order_id",
@@ -157,7 +157,7 @@ public class BeamJoinRelBoundedVsBoundedTest {
+ " o1.order_id=o2.site_id AND o2.price=o1.site_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.INTEGER, "order_id",
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
index 1dbd8b4..28ad99c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsBoundedTest.java
@@ -26,10 +26,10 @@ import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlOutputToConsoleFn;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.joda.time.Duration;
import org.junit.BeforeClass;
@@ -98,7 +98,7 @@ public class BeamJoinRelUnboundedVsBoundedTest {
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -124,7 +124,7 @@ public class BeamJoinRelUnboundedVsBoundedTest {
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -150,7 +150,7 @@ public class BeamJoinRelUnboundedVsBoundedTest {
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("helloworld")));
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
@@ -192,7 +192,7 @@ public class BeamJoinRelUnboundedVsBoundedTest {
+ " on "
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
index 5e5e416..a5a2e85 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRelUnboundedVsUnboundedTest.java
@@ -25,10 +25,10 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlOutputToConsoleFn;
import org.apache.beam.sdk.extensions.sql.mock.MockedUnboundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.joda.time.Duration;
import org.junit.BeforeClass;
@@ -88,7 +88,7 @@ public class BeamJoinRelUnboundedVsUnboundedTest {
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -121,7 +121,7 @@ public class BeamJoinRelUnboundedVsUnboundedTest {
// 2, 2 | 2, 5
// 3, 3 | NULL, NULL
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -151,7 +151,7 @@ public class BeamJoinRelUnboundedVsUnboundedTest {
+ " o1.order_id=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
TestUtils.RowsBuilder.of(
@@ -181,7 +181,7 @@ public class BeamJoinRelUnboundedVsUnboundedTest {
+ " o1.order_id1=o2.order_id"
;
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, beamSqlEnv);
rows.apply(ParDo.of(new BeamSqlOutputToConsoleFn("hello")));
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
index 9149dd4..425e554 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
@@ -23,9 +23,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -78,7 +78,7 @@ public class BeamMinusRelTest {
+ "SELECT order_id, site_id, price "
+ "FROM ORDER_DETAILS2 ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -100,7 +100,7 @@ public class BeamMinusRelTest {
+ "SELECT order_id, site_id, price "
+ "FROM ORDER_DETAILS2 ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).satisfies(new CheckSize(2));
PAssert.that(rows).containsInAnyOrder(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
index 36538c0..4de493a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBaseTest.java
@@ -25,11 +25,11 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -71,7 +71,7 @@ public class BeamSetOperatorRelBaseTest {
+ "FROM ORDER_DETAILS GROUP BY order_id, site_id"
+ ", TUMBLE(order_time, INTERVAL '1' HOUR) ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
// compare valueInString to ignore the windowStart & windowEnd
PAssert.that(rows.apply(ParDo.of(new TestUtils.BeamSqlRow2StringDoFn())))
.containsInAnyOrder(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
index 15e3b89..f033fa0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
@@ -24,9 +24,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.Before;
import org.junit.Rule;
@@ -78,7 +78,7 @@ public class BeamSortRelTest {
+ "FROM ORDER_DETAILS "
+ "ORDER BY order_id asc, site_id desc limit 4";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
Types.INTEGER, "site_id",
@@ -117,7 +117,7 @@ public class BeamSortRelTest {
+ "FROM ORDER_DETAILS "
+ "ORDER BY order_id asc, site_id desc NULLS FIRST limit 4";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -155,7 +155,7 @@ public class BeamSortRelTest {
+ "FROM ORDER_DETAILS "
+ "ORDER BY order_id asc, site_id desc NULLS LAST limit 4";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -178,7 +178,7 @@ public class BeamSortRelTest {
+ "FROM ORDER_DETAILS "
+ "ORDER BY order_id asc, site_id desc limit 4 offset 4";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -201,7 +201,7 @@ public class BeamSortRelTest {
+ "FROM ORDER_DETAILS "
+ "ORDER BY order_id asc, site_id desc limit 11";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
index c232b30..7cc52da 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
@@ -23,9 +23,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -63,7 +63,7 @@ public class BeamUnionRelTest {
+ " order_id, site_id, price "
+ "FROM ORDER_DETAILS ";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
@@ -86,7 +86,7 @@ public class BeamUnionRelTest {
+ " SELECT order_id, site_id, price "
+ "FROM ORDER_DETAILS";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.BIGINT, "order_id",
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
index e5fa864..ff31e55 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
@@ -23,9 +23,9 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlCli;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -60,7 +60,7 @@ public class BeamValuesRelTest {
public void testValues() throws Exception {
String sql = "insert into string_table(name, description) values "
+ "('hello', 'world'), ('james', 'bond')";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.VARCHAR, "name",
@@ -76,7 +76,7 @@ public class BeamValuesRelTest {
@Test
public void testValues_castInt() throws Exception {
String sql = "insert into int_table (c0, c1) values(cast(1 as int), cast(2 as int))";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.INTEGER, "c0",
@@ -91,7 +91,7 @@ public class BeamValuesRelTest {
@Test
public void testValues_onlySelect() throws Exception {
String sql = "select 1, '1'";
- PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+ PCollection<BeamRecord> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder.of(
Types.INTEGER, "EXPR$0",
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
index 8cdf2cd..7407a76 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/CheckSize.java
@@ -18,21 +18,21 @@
package org.apache.beam.sdk.extensions.sql.impl.rel;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.BeamRecord;
import org.junit.Assert;
/**
* Utility class to check size of BeamSQLRow iterable.
*/
-public class CheckSize implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
+public class CheckSize implements SerializableFunction<Iterable<BeamRecord>, Void> {
private int size;
public CheckSize(int size) {
this.size = size;
}
- @Override public Void apply(Iterable<BeamSqlRow> input) {
+ @Override public Void apply(Iterable<BeamRecord> input) {
int count = 0;
- for (BeamSqlRow row : input) {
+ for (BeamRecord row : input) {
count++;
}
Assert.assertEquals(size, count);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
index ffc6833..b58a17f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -32,11 +32,10 @@ import java.util.TimeZone;
import org.apache.beam.sdk.extensions.sql.BeamSql;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.calcite.util.Pair;
import org.junit.Rule;
@@ -62,8 +61,8 @@ public class BeamSqlBuiltinFunctionsIntegrationTestBase {
@Rule
public final TestPipeline pipeline = TestPipeline.create();
- protected PCollection<BeamSqlRow> getTestPCollection() {
- BeamSqlRowType type = BeamSqlRowType.create(
+ protected PCollection<BeamRecord> getTestPCollection() {
+ BeamSqlRecordType type = BeamSqlRecordType.create(
Arrays.asList("ts", "c_tinyint", "c_smallint",
"c_integer", "c_bigint", "c_float", "c_double", "c_decimal",
"c_tinyint_max", "c_smallint_max", "c_integer_max", "c_bigint_max"),
@@ -89,7 +88,7 @@ public class BeamSqlBuiltinFunctionsIntegrationTestBase {
9223372036854775807L
)
.buildIOReader(pipeline)
- .setCoder(new BeamSqlRowCoder(type));
+ .setCoder(type.getRecordCoder());
} catch (Exception e) {
throw new RuntimeException(e);
}
@@ -140,7 +139,7 @@ public class BeamSqlBuiltinFunctionsIntegrationTestBase {
* Build the corresponding SQL, compile to Beam Pipeline, run it, and check the result.
*/
public void buildRunAndCheck() {
- PCollection<BeamSqlRow> inputCollection = getTestPCollection();
+ PCollection<BeamRecord> inputCollection = getTestPCollection();
System.out.println("SQL:>\n" + getSql());
try {
List<String> names = new ArrayList<>();
@@ -153,10 +152,10 @@ public class BeamSqlBuiltinFunctionsIntegrationTestBase {
values.add(pair.getValue());
}
- PCollection<BeamSqlRow> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
+ PCollection<BeamRecord> rows = inputCollection.apply(BeamSql.simpleQuery(getSql()));
PAssert.that(rows).containsInAnyOrder(
TestUtils.RowsBuilder
- .of(BeamSqlRowType.create(names, types))
+ .of(BeamSqlRecordType.create(names, types))
.addRows(values)
.getRows()
);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
index 14de5b6..3569e31 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlComparisonOperatorsIntegrationTest.java
@@ -22,9 +22,8 @@ import java.math.BigDecimal;
import java.sql.Types;
import java.util.Arrays;
import org.apache.beam.sdk.extensions.sql.mock.MockedBoundedTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.Test;
@@ -282,8 +281,8 @@ public class BeamSqlComparisonOperatorsIntegrationTest
checker.buildRunAndCheck();
}
- @Override protected PCollection<BeamSqlRow> getTestPCollection() {
- BeamSqlRowType type = BeamSqlRowType.create(
+ @Override protected PCollection<BeamRecord> getTestPCollection() {
+ BeamSqlRecordType type = BeamSqlRecordType.create(
Arrays.asList(
"c_tinyint_0", "c_tinyint_1", "c_tinyint_2",
"c_smallint_0", "c_smallint_1", "c_smallint_2",
@@ -322,7 +321,7 @@ public class BeamSqlComparisonOperatorsIntegrationTest
false, true
)
.buildIOReader(pipeline)
- .setCoder(new BeamSqlRowCoder(type));
+ .setCoder(type.getRecordCoder());
} catch (Exception e) {
throw new RuntimeException(e);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
index 181c991..cda6a3c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -24,9 +24,9 @@ import static org.junit.Assert.assertTrue;
import java.util.Date;
import java.util.Iterator;
import org.apache.beam.sdk.extensions.sql.BeamSql;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.junit.Test;
@@ -63,17 +63,17 @@ public class BeamSqlDateFunctionsIntegrationTest
+ "CURRENT_TIMESTAMP as c3"
+ " FROM PCOLLECTION"
;
- PCollection<BeamSqlRow> rows = getTestPCollection().apply(
+ PCollection<BeamRecord> rows = getTestPCollection().apply(
BeamSql.simpleQuery(sql));
PAssert.that(rows).satisfies(new Checker());
pipeline.run();
}
- private static class Checker implements SerializableFunction<Iterable<BeamSqlRow>, Void> {
- @Override public Void apply(Iterable<BeamSqlRow> input) {
- Iterator<BeamSqlRow> iter = input.iterator();
+ private static class Checker implements SerializableFunction<Iterable<BeamRecord>, Void> {
+ @Override public Void apply(Iterable<BeamRecord> input) {
+ Iterator<BeamRecord> iter = input.iterator();
assertTrue(iter.hasNext());
- BeamSqlRow row = iter.next();
+ BeamRecord row = iter.next();
// LOCALTIME
Date date = new Date();
assertTrue(date.getTime() - row.getGregorianCalendar(0).getTime().getTime() < 1000);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
index c7c26eb..073ca52 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedBoundedTable.java
@@ -26,12 +26,12 @@ import java.util.List;
import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
@@ -41,11 +41,11 @@ import org.apache.beam.sdk.values.PDone;
*/
public class MockedBoundedTable extends MockedTable {
/** rows written to this table. */
- private static final ConcurrentLinkedQueue<BeamSqlRow> CONTENT = new ConcurrentLinkedQueue<>();
+ private static final ConcurrentLinkedQueue<BeamRecord> CONTENT = new ConcurrentLinkedQueue<>();
/** rows flow out from this table. */
- private final List<BeamSqlRow> rows = new ArrayList<>();
+ private final List<BeamRecord> rows = new ArrayList<>();
- public MockedBoundedTable(BeamSqlRowType beamSqlRowType) {
+ public MockedBoundedTable(BeamSqlRecordType beamSqlRowType) {
super(beamSqlRowType);
}
@@ -69,7 +69,7 @@ public class MockedBoundedTable extends MockedTable {
/**
* Build a mocked bounded table with the specified type.
*/
- public static MockedBoundedTable of(final BeamSqlRowType type) {
+ public static MockedBoundedTable of(final BeamSqlRecordType type) {
return new MockedBoundedTable(type);
}
@@ -88,7 +88,7 @@ public class MockedBoundedTable extends MockedTable {
* }</pre>
*/
public MockedBoundedTable addRows(Object... args) {
- List<BeamSqlRow> rows = buildRows(getRowType(), Arrays.asList(args));
+ List<BeamRecord> rows = buildRows(getRowType(), Arrays.asList(args));
this.rows.addAll(rows);
return this;
}
@@ -99,12 +99,12 @@ public class MockedBoundedTable extends MockedTable {
}
@Override
- public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+ public PCollection<BeamRecord> buildIOReader(Pipeline pipeline) {
return PBegin.in(pipeline).apply(
"MockedBoundedTable_Reader_" + COUNTER.incrementAndGet(), Create.of(rows));
}
- @Override public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+ @Override public PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter() {
return new OutputStore();
}
@@ -112,11 +112,11 @@ public class MockedBoundedTable extends MockedTable {
* Keep output in {@code CONTENT} for validation.
*
*/
- public static class OutputStore extends PTransform<PCollection<BeamSqlRow>, PDone> {
+ public static class OutputStore extends PTransform<PCollection<BeamRecord>, PDone> {
@Override
- public PDone expand(PCollection<BeamSqlRow> input) {
- input.apply(ParDo.of(new DoFn<BeamSqlRow, Void>() {
+ public PDone expand(PCollection<BeamRecord> input) {
+ input.apply(ParDo.of(new DoFn<BeamRecord, Void>() {
@ProcessElement
public void processElement(ProcessContext c) {
CONTENT.add(c.element());
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
index 6017ee7..59fc6e1 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedTable.java
@@ -20,9 +20,9 @@ package org.apache.beam.sdk.extensions.sql.mock;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
@@ -31,12 +31,12 @@ import org.apache.beam.sdk.values.PDone;
*/
public abstract class MockedTable extends BaseBeamTable {
public static final AtomicInteger COUNTER = new AtomicInteger();
- public MockedTable(BeamSqlRowType beamSqlRowType) {
+ public MockedTable(BeamSqlRecordType beamSqlRowType) {
super(beamSqlRowType);
}
@Override
- public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+ public PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter() {
throw new UnsupportedOperationException("buildIOWriter unsupported!");
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
index f9ea2ac..6194264 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/mock/MockedUnboundedTable.java
@@ -24,10 +24,9 @@ import java.util.List;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.extensions.sql.TestUtils;
import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TimestampedValue;
import org.apache.calcite.util.Pair;
@@ -39,10 +38,10 @@ import org.joda.time.Instant;
*/
public class MockedUnboundedTable extends MockedTable {
/** rows flow out from this table with the specified watermark instant. */
- private final List<Pair<Duration, List<BeamSqlRow>>> timestampedRows = new ArrayList<>();
+ private final List<Pair<Duration, List<BeamRecord>>> timestampedRows = new ArrayList<>();
/** specify the index of column in the row which stands for the event time field. */
private int timestampField;
- private MockedUnboundedTable(BeamSqlRowType beamSqlRowType) {
+ private MockedUnboundedTable(BeamSqlRecordType beamSqlRowType) {
super(beamSqlRowType);
}
@@ -83,7 +82,7 @@ public class MockedUnboundedTable extends MockedTable {
* }</pre>
*/
public MockedUnboundedTable addRows(Duration duration, Object... args) {
- List<BeamSqlRow> rows = TestUtils.buildRows(getRowType(), Arrays.asList(args));
+ List<BeamRecord> rows = TestUtils.buildRows(getRowType(), Arrays.asList(args));
// record the watermark + rows
this.timestampedRows.add(Pair.of(duration, rows));
return this;
@@ -93,11 +92,10 @@ public class MockedUnboundedTable extends MockedTable {
return BeamIOType.UNBOUNDED;
}
- @Override public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
- TestStream.Builder<BeamSqlRow> values = TestStream.create(
- new BeamSqlRowCoder(beamSqlRowType));
+ @Override public PCollection<BeamRecord> buildIOReader(Pipeline pipeline) {
+ TestStream.Builder<BeamRecord> values = TestStream.create(beamSqlRowType.getRecordCoder());
- for (Pair<Duration, List<BeamSqlRow>> pair : timestampedRows) {
+ for (Pair<Duration, List<BeamRecord>> pair : timestampedRows) {
values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey()));
for (int i = 0; i < pair.getValue().size(); i++) {
values = values.addElements(TimestampedValue.of(pair.getValue().get(i),
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
index ddff819..08f98c3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoderTest.java
@@ -21,8 +21,10 @@ package org.apache.beam.sdk.extensions.sql.schema;
import java.math.BigDecimal;
import java.util.Date;
import java.util.GregorianCalendar;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.testing.CoderProperties;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -57,10 +59,10 @@ public class BeamSqlRowCoderTest {
}
};
- BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(
+ BeamSqlRecordType beamSQLRowType = CalciteUtils.toBeamRowType(
protoRowType.apply(new JavaTypeFactoryImpl(
RelDataTypeSystem.DEFAULT)));
- BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+ BeamRecord row = new BeamRecord(beamSQLRowType);
row.addField("col_tinyint", Byte.valueOf("1"));
row.addField("col_smallint", Short.valueOf("1"));
row.addField("col_integer", 1);
@@ -76,7 +78,7 @@ public class BeamSqlRowCoderTest {
row.addField("col_boolean", true);
- BeamSqlRowCoder coder = new BeamSqlRowCoder(beamSQLRowType);
+ BeamRecordCoder coder = beamSQLRowType.getRecordCoder();
CoderProperties.coderDecodeEncodeEqual(coder, row);
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
index 05af36c..2fc013d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTableTest.java
@@ -21,13 +21,13 @@ package org.apache.beam.sdk.extensions.sql.schema.kafka;
import java.io.Serializable;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.calcite.rel.type.RelDataType;
@@ -45,8 +45,8 @@ import org.junit.Test;
public class BeamKafkaCSVTableTest {
@Rule
public TestPipeline pipeline = TestPipeline.create();
- public static BeamSqlRow row1 = new BeamSqlRow(genRowType());
- public static BeamSqlRow row2 = new BeamSqlRow(genRowType());
+ public static BeamRecord row1 = new BeamRecord(genRowType());
+ public static BeamRecord row2 = new BeamRecord(genRowType());
@BeforeClass
public static void setUp() {
@@ -60,7 +60,7 @@ public class BeamKafkaCSVTableTest {
}
@Test public void testCsvRecorderDecoder() throws Exception {
- PCollection<BeamSqlRow> result = pipeline
+ PCollection<BeamRecord> result = pipeline
.apply(
Create.of("1,\"1\",1.0", "2,2,2.0")
)
@@ -75,7 +75,7 @@ public class BeamKafkaCSVTableTest {
}
@Test public void testCsvRecorderEncoder() throws Exception {
- PCollection<BeamSqlRow> result = pipeline
+ PCollection<BeamRecord> result = pipeline
.apply(
Create.of(row1, row2)
)
@@ -90,7 +90,7 @@ public class BeamKafkaCSVTableTest {
pipeline.run();
}
- private static BeamSqlRowType genRowType() {
+ private static BeamSqlRecordType genRowType() {
return CalciteUtils.toBeamRowType(new RelProtoDataType() {
@Override public RelDataType apply(RelDataTypeFactory a0) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
index 79e3d6d..4a39f7c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableTest.java
@@ -33,10 +33,10 @@ import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -69,7 +69,7 @@ public class BeamTextCSVTableTest {
private static Object[] data2 = new Object[] { 2, 2L, 2.2F, 2.2, "bond" };
private static List<Object[]> testData = Arrays.asList(data1, data2);
- private static List<BeamSqlRow> testDataRows = new ArrayList<BeamSqlRow>() {{
+ private static List<BeamRecord> testDataRows = new ArrayList<BeamRecord>() {{
for (Object[] data : testData) {
add(buildRow(data));
}
@@ -80,7 +80,7 @@ public class BeamTextCSVTableTest {
private static File writerTargetFile;
@Test public void testBuildIOReader() {
- PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+ PCollection<BeamRecord> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
readerSourceFile.getAbsolutePath()).buildIOReader(pipeline);
PAssert.that(rows).containsInAnyOrder(testDataRows);
pipeline.run();
@@ -93,7 +93,7 @@ public class BeamTextCSVTableTest {
.buildIOWriter());
pipeline.run();
- PCollection<BeamSqlRow> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
+ PCollection<BeamRecord> rows = new BeamTextCSVTable(buildBeamSqlRowType(),
writerTargetFile.getAbsolutePath()).buildIOReader(pipeline2);
// confirm the two reads match
@@ -166,11 +166,11 @@ public class BeamTextCSVTableTest {
.add("amount", SqlTypeName.DOUBLE).add("user_name", SqlTypeName.VARCHAR).build();
}
- private static BeamSqlRowType buildBeamSqlRowType() {
+ private static BeamSqlRecordType buildBeamSqlRowType() {
return CalciteUtils.toBeamRowType(buildRelDataType());
}
- private static BeamSqlRow buildRow(Object[] data) {
- return new BeamSqlRow(buildBeamSqlRowType(), Arrays.asList(data));
+ private static BeamRecord buildRow(Object[] data) {
+ return new BeamRecord(buildBeamSqlRowType(), Arrays.asList(data));
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
index 821abc9..dca2ad7 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamAggregationTransformTest.java
@@ -21,14 +21,13 @@ import java.text.ParseException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Combine;
@@ -36,6 +35,7 @@ import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.GroupByKey;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.calcite.rel.core.AggregateCall;
@@ -63,14 +63,14 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
private List<AggregateCall> aggCalls;
- private BeamSqlRowType keyType;
- private BeamSqlRowType aggPartType;
- private BeamSqlRowType outputType;
+ private BeamSqlRecordType keyType;
+ private BeamSqlRecordType aggPartType;
+ private BeamSqlRecordType outputType;
- private BeamSqlRowCoder inRecordCoder;
- private BeamSqlRowCoder keyCoder;
- private BeamSqlRowCoder aggCoder;
- private BeamSqlRowCoder outRecordCoder;
+ private BeamRecordCoder inRecordCoder;
+ private BeamRecordCoder keyCoder;
+ private BeamRecordCoder aggCoder;
+ private BeamRecordCoder outRecordCoder;
/**
* This step equals to below query.
@@ -99,28 +99,28 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
public void testCountPerElementBasic() throws ParseException {
setupEnvironment();
- PCollection<BeamSqlRow> input = p.apply(Create.of(inputRows));
+ PCollection<BeamRecord> input = p.apply(Create.of(inputRows));
//1. extract fields in group-by key part
- PCollection<KV<BeamSqlRow, BeamSqlRow>> exGroupByStream = input.apply("exGroupBy",
+ PCollection<KV<BeamRecord, BeamRecord>> exGroupByStream = input.apply("exGroupBy",
WithKeys
.of(new BeamAggregationTransforms.AggregationGroupByKeyFn(-1, ImmutableBitSet.of(0))))
- .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, inRecordCoder));
+ .setCoder(KvCoder.<BeamRecord, BeamRecord>of(keyCoder, inRecordCoder));
//2. apply a GroupByKey.
- PCollection<KV<BeamSqlRow, Iterable<BeamSqlRow>>> groupedStream = exGroupByStream
- .apply("groupBy", GroupByKey.<BeamSqlRow, BeamSqlRow>create())
- .setCoder(KvCoder.<BeamSqlRow, Iterable<BeamSqlRow>>of(keyCoder,
- IterableCoder.<BeamSqlRow>of(inRecordCoder)));
+ PCollection<KV<BeamRecord, Iterable<BeamRecord>>> groupedStream = exGroupByStream
+ .apply("groupBy", GroupByKey.<BeamRecord, BeamRecord>create())
+ .setCoder(KvCoder.<BeamRecord, Iterable<BeamRecord>>of(keyCoder,
+ IterableCoder.<BeamRecord>of(inRecordCoder)));
//3. run aggregation functions
- PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = groupedStream.apply("aggregation",
- Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>groupedValues(
+ PCollection<KV<BeamRecord, BeamRecord>> aggregatedStream = groupedStream.apply("aggregation",
+ Combine.<BeamRecord, BeamRecord, BeamRecord>groupedValues(
new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputRowType)))
- .setCoder(KvCoder.<BeamSqlRow, BeamSqlRow>of(keyCoder, aggCoder));
+ .setCoder(KvCoder.<BeamRecord, BeamRecord>of(keyCoder, aggCoder));
//4. flat KV to a single record
- PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply("mergeRecord",
+ PCollection<BeamRecord> mergedStream = aggregatedStream.apply("mergeRecord",
ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, aggCalls, -1)));
mergedStream.setCoder(outRecordCoder);
@@ -332,10 +332,10 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
* Coders used in aggregation steps.
*/
private void prepareTypeAndCoder() {
- inRecordCoder = new BeamSqlRowCoder(inputRowType);
+ inRecordCoder = inputRowType.getRecordCoder();
keyType = initTypeOfSqlRow(Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER)));
- keyCoder = new BeamSqlRowCoder(keyType);
+ keyCoder = keyType.getRecordCoder();
aggPartType = initTypeOfSqlRow(
Arrays.asList(KV.of("count", SqlTypeName.BIGINT),
@@ -360,35 +360,35 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
KV.of("sum8", SqlTypeName.INTEGER), KV.of("avg8", SqlTypeName.INTEGER),
KV.of("max8", SqlTypeName.INTEGER), KV.of("min8", SqlTypeName.INTEGER)
));
- aggCoder = new BeamSqlRowCoder(aggPartType);
+ aggCoder = aggPartType.getRecordCoder();
outputType = prepareFinalRowType();
- outRecordCoder = new BeamSqlRowCoder(outputType);
+ outRecordCoder = outputType.getRecordCoder();
}
/**
* expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}.
*/
- private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationGroupByKeyFn() {
+ private List<KV<BeamRecord, BeamRecord>> prepareResultOfAggregationGroupByKeyFn() {
return Arrays.asList(
- KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+ KV.of(new BeamRecord(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
inputRows.get(0)),
- KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
+ KV.of(new BeamRecord(keyType, Arrays.<Object>asList(inputRows.get(1).getInteger(0))),
inputRows.get(1)),
- KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
+ KV.of(new BeamRecord(keyType, Arrays.<Object>asList(inputRows.get(2).getInteger(0))),
inputRows.get(2)),
- KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
+ KV.of(new BeamRecord(keyType, Arrays.<Object>asList(inputRows.get(3).getInteger(0))),
inputRows.get(3)));
}
/**
* expected results after {@link BeamAggregationTransforms.AggregationCombineFn}.
*/
- private List<KV<BeamSqlRow, BeamSqlRow>> prepareResultOfAggregationCombineFn()
+ private List<KV<BeamRecord, BeamRecord>> prepareResultOfAggregationCombineFn()
throws ParseException {
return Arrays.asList(
- KV.of(new BeamSqlRow(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
- new BeamSqlRow(aggPartType, Arrays.<Object>asList(
+ KV.of(new BeamRecord(keyType, Arrays.<Object>asList(inputRows.get(0).getInteger(0))),
+ new BeamRecord(aggPartType, Arrays.<Object>asList(
4L,
10000L, 2500L, 4000L, 1000L,
(short) 10, (short) 2, (short) 4, (short) 1,
@@ -404,7 +404,7 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
/**
* Row type of final output row.
*/
- private BeamSqlRowType prepareFinalRowType() {
+ private BeamSqlRecordType prepareFinalRowType() {
FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
List<KV<String, SqlTypeName>> columnMetadata =
Arrays.asList(KV.of("f_int", SqlTypeName.INTEGER), KV.of("count", SqlTypeName.BIGINT),
@@ -438,8 +438,8 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest{
/**
* expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}.
*/
- private BeamSqlRow prepareResultOfMergeAggregationRecord() throws ParseException {
- return new BeamSqlRow(outputType, Arrays.<Object>asList(
+ private BeamRecord prepareResultOfMergeAggregationRecord() throws ParseException {
+ return new BeamRecord(outputType, Arrays.<Object>asList(
1, 4L,
10000L, 2500L, 4000L, 1000L,
(short) 10, (short) 2, (short) 4, (short) 1,
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
index af7ec23..e31463b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/schema/transform/BeamTransformBaseTest.java
@@ -24,8 +24,8 @@ import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -38,8 +38,8 @@ import org.junit.BeforeClass;
public class BeamTransformBaseTest {
public static DateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
- public static BeamSqlRowType inputRowType;
- public static List<BeamSqlRow> inputRows;
+ public static BeamSqlRecordType inputRowType;
+ public static List<BeamRecord> inputRows;
@BeforeClass
public static void prepareInput() throws NumberFormatException, ParseException{
@@ -68,7 +68,7 @@ public class BeamTransformBaseTest {
/**
* create a {@code BeamSqlRowType} for given column metadata.
*/
- public static BeamSqlRowType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
+ public static BeamSqlRecordType initTypeOfSqlRow(List<KV<String, SqlTypeName>> columnMetadata){
FieldInfoBuilder builder = BeamQueryPlanner.TYPE_FACTORY.builder();
for (KV<String, SqlTypeName> cm : columnMetadata) {
builder.add(cm.getKey(), cm.getValue());
@@ -79,7 +79,7 @@ public class BeamTransformBaseTest {
/**
* Create an empty row with given column metadata.
*/
- public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
+ public static BeamRecord initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata) {
return initBeamSqlRow(columnMetadata, Arrays.asList());
}
@@ -87,11 +87,11 @@ public class BeamTransformBaseTest {
* Create a row with given column metadata, and values for each column.
*
*/
- public static BeamSqlRow initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
+ public static BeamRecord initBeamSqlRow(List<KV<String, SqlTypeName>> columnMetadata,
List<Object> rowValues){
- BeamSqlRowType rowType = initTypeOfSqlRow(columnMetadata);
+ BeamSqlRecordType rowType = initTypeOfSqlRow(columnMetadata);
- return new BeamSqlRow(rowType, rowValues);
+ return new BeamRecord(rowType, rowValues);
}
}
[4/7] beam git commit: refactor BeamRecord, BeamRecordType,
BeamSqlRecordType, BeamRecordCoder
Posted by ro...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordHelper.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordHelper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordHelper.java
new file mode 100644
index 0000000..b910c84
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordHelper.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.schema;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.values.BeamRecord;
+
+/**
+ * A {@link Coder} encodes {@link BeamRecord}.
+ */
+@Experimental
+public class BeamSqlRecordHelper {
+
+ public static BeamSqlRecordType getSqlRecordType(BeamRecord record) {
+ return (BeamSqlRecordType) record.getDataType();
+ }
+
+ /**
+ * {@link Coder} for Java type {@link Short}.
+ */
+ public static class ShortCoder extends CustomCoder<Short> {
+ private static final ShortCoder INSTANCE = new ShortCoder();
+
+ public static ShortCoder of() {
+ return INSTANCE;
+ }
+
+ private ShortCoder() {
+ }
+
+ @Override
+ public void encode(Short value, OutputStream outStream) throws CoderException, IOException {
+ new DataOutputStream(outStream).writeShort(value);
+ }
+
+ @Override
+ public Short decode(InputStream inStream) throws CoderException, IOException {
+ return new DataInputStream(inStream).readShort();
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+ /**
+ * {@link Coder} for Java type {@link Float}, it's stored as {@link BigDecimal}.
+ */
+ public static class FloatCoder extends CustomCoder<Float> {
+ private static final FloatCoder INSTANCE = new FloatCoder();
+ private static final BigDecimalCoder CODER = BigDecimalCoder.of();
+
+ public static FloatCoder of() {
+ return INSTANCE;
+ }
+
+ private FloatCoder() {
+ }
+
+ @Override
+ public void encode(Float value, OutputStream outStream) throws CoderException, IOException {
+ CODER.encode(new BigDecimal(value), outStream);
+ }
+
+ @Override
+ public Float decode(InputStream inStream) throws CoderException, IOException {
+ return CODER.decode(inStream).floatValue();
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+ /**
+ * {@link Coder} for Java type {@link Double}, it's stored as {@link BigDecimal}.
+ */
+ public static class DoubleCoder extends CustomCoder<Double> {
+ private static final DoubleCoder INSTANCE = new DoubleCoder();
+ private static final BigDecimalCoder CODER = BigDecimalCoder.of();
+
+ public static DoubleCoder of() {
+ return INSTANCE;
+ }
+
+ private DoubleCoder() {
+ }
+
+ @Override
+ public void encode(Double value, OutputStream outStream) throws CoderException, IOException {
+ CODER.encode(new BigDecimal(value), outStream);
+ }
+
+ @Override
+ public Double decode(InputStream inStream) throws CoderException, IOException {
+ return CODER.decode(inStream).doubleValue();
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+
+ /**
+ * {@link Coder} for Java type {@link GregorianCalendar}, it's stored as {@link Long}.
+ */
+ public static class TimeCoder extends CustomCoder<GregorianCalendar> {
+ private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
+ private static final TimeCoder INSTANCE = new TimeCoder();
+
+ public static TimeCoder of() {
+ return INSTANCE;
+ }
+
+ private TimeCoder() {
+ }
+
+ @Override
+ public void encode(GregorianCalendar value, OutputStream outStream)
+ throws CoderException, IOException {
+ longCoder.encode(value.getTime().getTime(), outStream);
+ }
+
+ @Override
+ public GregorianCalendar decode(InputStream inStream) throws CoderException, IOException {
+ GregorianCalendar calendar = new GregorianCalendar();
+ calendar.setTime(new Date(longCoder.decode(inStream)));
+ return calendar;
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+ /**
+ * {@link Coder} for Java type {@link Date}, it's stored as {@link Long}.
+ */
+ public static class DateCoder extends CustomCoder<Date> {
+ private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
+ private static final DateCoder INSTANCE = new DateCoder();
+
+ public static DateCoder of() {
+ return INSTANCE;
+ }
+
+ private DateCoder() {
+ }
+
+ @Override
+ public void encode(Date value, OutputStream outStream) throws CoderException, IOException {
+ longCoder.encode(value.getTime(), outStream);
+ }
+
+ @Override
+ public Date decode(InputStream inStream) throws CoderException, IOException {
+ return new Date(longCoder.decode(inStream));
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+
+ /**
+ * {@link Coder} for Java type {@link Boolean}.
+ */
+ public static class BooleanCoder extends CustomCoder<Boolean> {
+ private static final BooleanCoder INSTANCE = new BooleanCoder();
+
+ public static BooleanCoder of() {
+ return INSTANCE;
+ }
+
+ private BooleanCoder() {
+ }
+
+ @Override
+ public void encode(Boolean value, OutputStream outStream) throws CoderException, IOException {
+ new DataOutputStream(outStream).writeBoolean(value);
+ }
+
+ @Override
+ public Boolean decode(InputStream inStream) throws CoderException, IOException {
+ return new DataInputStream(inStream).readBoolean();
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordType.java
new file mode 100644
index 0000000..b295049
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRecordType.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.schema;
+
+import java.math.BigDecimal;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.coders.BigDecimalCoder;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.ByteCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.BooleanCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.DateCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.DoubleCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.FloatCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.ShortCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper.TimeCoder;
+import org.apache.beam.sdk.values.BeamRecord;
+import org.apache.beam.sdk.values.BeamRecordType;
+
+/**
+ * Type provider for {@link BeamRecord} with SQL types.
+ *
+ * <p>Limited SQL types are supported now, visit
+ * <a href="https://beam.apache.org/blog/2017/07/21/sql-dsl.html#data-type">data types</a>
+ * for more details.
+ *
+ */
+public class BeamSqlRecordType extends BeamRecordType {
+ private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
+ static {
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
+
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
+ SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
+ }
+
+ public List<Integer> fieldsType;
+
+ protected BeamSqlRecordType(List<String> fieldsName, List<Coder> fieldsCoder) {
+ super(fieldsName, fieldsCoder);
+ }
+
+ private BeamSqlRecordType(List<String> fieldsName, List<Integer> fieldsType
+ , List<Coder> fieldsCoder) {
+ super(fieldsName, fieldsCoder);
+ this.fieldsType = fieldsType;
+ }
+
+ public static BeamSqlRecordType create(List<String> fieldNames,
+ List<Integer> fieldTypes) {
+ List<Coder> fieldCoders = new ArrayList<>();
+ for (int idx = 0; idx < fieldTypes.size(); ++idx) {
+ switch (fieldTypes.get(idx)) {
+ case Types.INTEGER:
+ fieldCoders.add(BigEndianIntegerCoder.of());
+ break;
+ case Types.SMALLINT:
+ fieldCoders.add(ShortCoder.of());
+ break;
+ case Types.TINYINT:
+ fieldCoders.add(ByteCoder.of());
+ break;
+ case Types.DOUBLE:
+ fieldCoders.add(DoubleCoder.of());
+ break;
+ case Types.FLOAT:
+ fieldCoders.add(FloatCoder.of());
+ break;
+ case Types.DECIMAL:
+ fieldCoders.add(BigDecimalCoder.of());
+ break;
+ case Types.BIGINT:
+ fieldCoders.add(BigEndianLongCoder.of());
+ break;
+ case Types.VARCHAR:
+ case Types.CHAR:
+ fieldCoders.add(StringUtf8Coder.of());
+ break;
+ case Types.TIME:
+ fieldCoders.add(TimeCoder.of());
+ break;
+ case Types.DATE:
+ case Types.TIMESTAMP:
+ fieldCoders.add(DateCoder.of());
+ break;
+ case Types.BOOLEAN:
+ fieldCoders.add(BooleanCoder.of());
+ break;
+
+ default:
+ throw new UnsupportedOperationException(
+ "Data type: " + fieldTypes.get(idx) + " not supported yet!");
+ }
+ }
+ return new BeamSqlRecordType(fieldNames, fieldTypes, fieldCoders);
+ }
+
+ @Override
+ public void validateValueType(int index, Object fieldValue) throws IllegalArgumentException {
+ int fieldType = fieldsType.get(index);
+ Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(fieldType);
+ if (javaClazz == null) {
+ throw new IllegalArgumentException("Data type: " + fieldType + " not supported yet!");
+ }
+
+ if (!fieldValue.getClass().equals(javaClazz)) {
+ throw new IllegalArgumentException(
+ String.format("[%s](%s) doesn't match type [%s]",
+ fieldValue, fieldValue.getClass(), fieldType)
+ );
+ }
+ }
+
+ public List<Integer> getFieldsType() {
+ return fieldsType;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj != null && obj instanceof BeamSqlRecordType) {
+ BeamSqlRecordType ins = (BeamSqlRecordType) obj;
+ return fieldsType.equals(ins.getFieldsType()) && getFieldsName().equals(ins.getFieldsName());
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * getFieldsName().hashCode() + getFieldsType().hashCode();
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.java
deleted file mode 100644
index cb5c7ea..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRow.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.beam.sdk.extensions.sql.schema;
-
-import java.util.List;
-import org.apache.beam.sdk.values.BeamRecord;
-import org.apache.beam.sdk.values.PCollection;
-
-/**
- * {@link BeamSqlRow} represents one row element in a {@link PCollection},
- * with type provider {@link BeamSqlRowType}.
- */
-public class BeamSqlRow extends BeamRecord {
- public BeamSqlRow(BeamSqlRowType dataType, List<Object> dataValues) {
- super(dataType, dataValues);
- }
-
- public BeamSqlRow(BeamSqlRowType dataType) {
- super(dataType);
- }
-
- @Override
- public BeamSqlRowType getDataType() {
- return (BeamSqlRowType) super.getDataType();
- }
-}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
deleted file mode 100644
index c7656af..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
+++ /dev/null
@@ -1,186 +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.beam.sdk.extensions.sql.schema;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.sql.Types;
-import java.util.BitSet;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import org.apache.beam.sdk.coders.BigDecimalCoder;
-import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
-import org.apache.beam.sdk.coders.BigEndianLongCoder;
-import org.apache.beam.sdk.coders.BitSetCoder;
-import org.apache.beam.sdk.coders.ByteCoder;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.CoderException;
-import org.apache.beam.sdk.coders.CustomCoder;
-import org.apache.beam.sdk.coders.DoubleCoder;
-import org.apache.beam.sdk.coders.InstantCoder;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-
-/**
- * A {@link Coder} encodes {@link BeamSqlRow}.
- */
-public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
- private BeamSqlRowType sqlRecordType;
-
- private static final BitSetCoder nullListCoder = BitSetCoder.of();
-
- private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
- private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
- private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of();
- private static final DoubleCoder doubleCoder = DoubleCoder.of();
- private static final InstantCoder instantCoder = InstantCoder.of();
- private static final BigDecimalCoder bigDecimalCoder = BigDecimalCoder.of();
- private static final ByteCoder byteCoder = ByteCoder.of();
-
- public BeamSqlRowCoder(BeamSqlRowType sqlRecordType) {
- this.sqlRecordType = sqlRecordType;
- }
-
- @Override
- public void encode(BeamSqlRow value, OutputStream outStream)
- throws CoderException, IOException {
- nullListCoder.encode(value.getNullFields(), outStream);
- for (int idx = 0; idx < value.size(); ++idx) {
- if (value.getNullFields().get(idx)) {
- continue;
- }
-
- switch (sqlRecordType.getFieldsType().get(idx)) {
- case Types.INTEGER:
- intCoder.encode(value.getInteger(idx), outStream);
- break;
- case Types.SMALLINT:
- intCoder.encode((int) value.getShort(idx), outStream);
- break;
- case Types.TINYINT:
- byteCoder.encode(value.getByte(idx), outStream);
- break;
- case Types.DOUBLE:
- doubleCoder.encode(value.getDouble(idx), outStream);
- break;
- case Types.FLOAT:
- doubleCoder.encode((double) value.getFloat(idx), outStream);
- break;
- case Types.DECIMAL:
- bigDecimalCoder.encode(value.getBigDecimal(idx), outStream);
- break;
- case Types.BIGINT:
- longCoder.encode(value.getLong(idx), outStream);
- break;
- case Types.VARCHAR:
- case Types.CHAR:
- stringCoder.encode(value.getString(idx), outStream);
- break;
- case Types.TIME:
- longCoder.encode(value.getGregorianCalendar(idx).getTime().getTime(), outStream);
- break;
- case Types.DATE:
- case Types.TIMESTAMP:
- longCoder.encode(value.getDate(idx).getTime(), outStream);
- break;
- case Types.BOOLEAN:
- byteCoder.encode((byte) (value.getBoolean(idx) ? 1 : 0), outStream);
- break;
-
- default:
- throw new UnsupportedOperationException(
- "Data type: " + sqlRecordType.getFieldsType().get(idx) + " not supported yet!");
- }
- }
-
- instantCoder.encode(value.getWindowStart(), outStream);
- instantCoder.encode(value.getWindowEnd(), outStream);
- }
-
- @Override
- public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
- BitSet nullFields = nullListCoder.decode(inStream);
-
- BeamSqlRow record = new BeamSqlRow(sqlRecordType);
- record.setNullFields(nullFields);
- for (int idx = 0; idx < sqlRecordType.size(); ++idx) {
- if (nullFields.get(idx)) {
- continue;
- }
-
- switch (sqlRecordType.getFieldsType().get(idx)) {
- case Types.INTEGER:
- record.addField(idx, intCoder.decode(inStream));
- break;
- case Types.SMALLINT:
- record.addField(idx, intCoder.decode(inStream).shortValue());
- break;
- case Types.TINYINT:
- record.addField(idx, byteCoder.decode(inStream));
- break;
- case Types.DOUBLE:
- record.addField(idx, doubleCoder.decode(inStream));
- break;
- case Types.FLOAT:
- record.addField(idx, doubleCoder.decode(inStream).floatValue());
- break;
- case Types.BIGINT:
- record.addField(idx, longCoder.decode(inStream));
- break;
- case Types.DECIMAL:
- record.addField(idx, bigDecimalCoder.decode(inStream));
- break;
- case Types.VARCHAR:
- case Types.CHAR:
- record.addField(idx, stringCoder.decode(inStream));
- break;
- case Types.TIME:
- GregorianCalendar calendar = new GregorianCalendar();
- calendar.setTime(new Date(longCoder.decode(inStream)));
- record.addField(idx, calendar);
- break;
- case Types.DATE:
- case Types.TIMESTAMP:
- record.addField(idx, new Date(longCoder.decode(inStream)));
- break;
- case Types.BOOLEAN:
- record.addField(idx, byteCoder.decode(inStream) == 1);
- break;
-
- default:
- throw new UnsupportedOperationException("Data type: "
- + sqlRecordType.getFieldsType().get(idx)
- + " not supported yet!");
- }
- }
-
- record.setWindowStart(instantCoder.decode(inStream));
- record.setWindowEnd(instantCoder.decode(inStream));
-
- return record;
- }
-
- public BeamSqlRowType getSqlRecordType() {
- return sqlRecordType;
- }
-
- @Override
- public void verifyDeterministic()
- throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
- }
-}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.java
deleted file mode 100644
index 7584dad..0000000
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowType.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.beam.sdk.extensions.sql.schema;
-
-import java.math.BigDecimal;
-import java.sql.Types;
-import java.util.Date;
-import java.util.GregorianCalendar;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.beam.sdk.values.BeamRecordTypeProvider;
-
-/**
- * Type provider for {@link BeamSqlRow} with SQL types.
- *
- * <p>Limited SQL types are supported now, visit
- * <a href="https://beam.apache.org/blog/2017/07/21/sql-dsl.html#data-type">data types</a>
- * for more details.
- *
- */
-public class BeamSqlRowType extends BeamRecordTypeProvider {
- private static final Map<Integer, Class> SQL_TYPE_TO_JAVA_CLASS = new HashMap<>();
- static {
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TINYINT, Byte.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.SMALLINT, Short.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.INTEGER, Integer.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.BIGINT, Long.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.FLOAT, Float.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DOUBLE, Double.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DECIMAL, BigDecimal.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.BOOLEAN, Boolean.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.CHAR, String.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.VARCHAR, String.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TIME, GregorianCalendar.class);
-
- SQL_TYPE_TO_JAVA_CLASS.put(Types.DATE, Date.class);
- SQL_TYPE_TO_JAVA_CLASS.put(Types.TIMESTAMP, Date.class);
- }
-
- public List<Integer> fieldsType;
-
- protected BeamSqlRowType(List<String> fieldsName) {
- super(fieldsName);
- }
-
- public BeamSqlRowType(List<String> fieldsName, List<Integer> fieldsType) {
- super(fieldsName);
- this.fieldsType = fieldsType;
- }
-
- public static BeamSqlRowType create(List<String> fieldNames,
- List<Integer> fieldTypes) {
- return new BeamSqlRowType(fieldNames, fieldTypes);
- }
-
- @Override
- public void validateValueType(int index, Object fieldValue) throws IllegalArgumentException {
- int fieldType = fieldsType.get(index);
- Class javaClazz = SQL_TYPE_TO_JAVA_CLASS.get(fieldType);
- if (javaClazz == null) {
- throw new IllegalArgumentException("Data type: " + fieldType + " not supported yet!");
- }
-
- if (!fieldValue.getClass().equals(javaClazz)) {
- throw new IllegalArgumentException(
- String.format("[%s](%s) doesn't match type [%s]",
- fieldValue, fieldValue.getClass(), fieldType)
- );
- }
- }
-
- public List<Integer> getFieldsType() {
- return fieldsType;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj != null && obj instanceof BeamSqlRowType) {
- BeamSqlRowType ins = (BeamSqlRowType) obj;
- return fieldsType.equals(ins.getFieldsType()) && getFieldsName().equals(ins.getFieldsName());
- } else {
- return false;
- }
- }
-
- @Override
- public int hashCode() {
- return 31 * getFieldsName().hashCode() + getFieldsType().hashCode();
- }
-}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
index c179935..b370d9d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlTable.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.extensions.sql.schema;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
@@ -37,16 +38,16 @@ public interface BeamSqlTable {
* create a {@code PCollection<BeamSqlRow>} from source.
*
*/
- PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline);
+ PCollection<BeamRecord> buildIOReader(Pipeline pipeline);
/**
* create a {@code IO.write()} instance to write to target.
*
*/
- PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter();
+ PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter();
/**
* Get the schema info of the table.
*/
- BeamSqlRowType getRowType();
+ BeamSqlRecordType getRowType();
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
index c769928..63c9720 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamTableUtils.java
@@ -23,6 +23,7 @@ import java.io.StringReader;
import java.io.StringWriter;
import java.math.BigDecimal;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.NlsString;
import org.apache.commons.csv.CSVFormat;
@@ -34,11 +35,11 @@ import org.apache.commons.csv.CSVRecord;
* Utility methods for working with {@code BeamTable}.
*/
public final class BeamTableUtils {
- public static BeamSqlRow csvLine2BeamSqlRow(
+ public static BeamRecord csvLine2BeamSqlRow(
CSVFormat csvFormat,
String line,
- BeamSqlRowType beamSqlRowType) {
- BeamSqlRow row = new BeamSqlRow(beamSqlRowType);
+ BeamSqlRecordType beamSqlRowType) {
+ BeamRecord row = new BeamRecord(beamSqlRowType);
try (StringReader reader = new StringReader(line)) {
CSVParser parser = csvFormat.parse(reader);
CSVRecord rawRecord = parser.getRecords().get(0);
@@ -60,7 +61,7 @@ public final class BeamTableUtils {
return row;
}
- public static String beamSqlRow2CsvLine(BeamSqlRow row, CSVFormat csvFormat) {
+ public static String beamSqlRow2CsvLine(BeamRecord row, CSVFormat csvFormat) {
StringWriter writer = new StringWriter();
try (CSVPrinter printer = csvFormat.print(writer)) {
for (int i = 0; i < row.size(); i++) {
@@ -73,13 +74,14 @@ public final class BeamTableUtils {
return writer.toString();
}
- public static void addFieldWithAutoTypeCasting(BeamSqlRow row, int idx, Object rawObj) {
+ public static void addFieldWithAutoTypeCasting(BeamRecord row, int idx, Object rawObj) {
if (rawObj == null) {
row.addField(idx, null);
return;
}
- SqlTypeName columnType = CalciteUtils.getFieldType(row.getDataType(), idx);
+ SqlTypeName columnType = CalciteUtils.getFieldType(BeamSqlRecordHelper.getSqlRecordType(row)
+ , idx);
// auto-casting for numberics
if ((rawObj instanceof String && SqlTypeName.NUMERIC_TYPES.contains(columnType))
|| (rawObj instanceof BigDecimal && columnType != SqlTypeName.DECIMAL)) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
index 2a50947..f137379 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaCSVTable.java
@@ -18,12 +18,12 @@
package org.apache.beam.sdk.extensions.sql.schema.kafka;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.commons.csv.CSVFormat;
@@ -34,45 +34,45 @@ import org.apache.commons.csv.CSVFormat;
*/
public class BeamKafkaCSVTable extends BeamKafkaTable {
private CSVFormat csvFormat;
- public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+ public BeamKafkaCSVTable(BeamSqlRecordType beamSqlRowType, String bootstrapServers,
List<String> topics) {
this(beamSqlRowType, bootstrapServers, topics, CSVFormat.DEFAULT);
}
- public BeamKafkaCSVTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+ public BeamKafkaCSVTable(BeamSqlRecordType beamSqlRowType, String bootstrapServers,
List<String> topics, CSVFormat format) {
super(beamSqlRowType, bootstrapServers, topics);
this.csvFormat = format;
}
@Override
- public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+ public PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamRecord>>
getPTransformForInput() {
return new CsvRecorderDecoder(beamSqlRowType, csvFormat);
}
@Override
- public PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+ public PTransform<PCollection<BeamRecord>, PCollection<KV<byte[], byte[]>>>
getPTransformForOutput() {
return new CsvRecorderEncoder(beamSqlRowType, csvFormat);
}
/**
- * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamSqlRow}.
+ * A PTransform to convert {@code KV<byte[], byte[]>} to {@link BeamRecord}.
*
*/
public static class CsvRecorderDecoder
- extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>> {
- private BeamSqlRowType rowType;
+ extends PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamRecord>> {
+ private BeamSqlRecordType rowType;
private CSVFormat format;
- public CsvRecorderDecoder(BeamSqlRowType rowType, CSVFormat format) {
+ public CsvRecorderDecoder(BeamSqlRecordType rowType, CSVFormat format) {
this.rowType = rowType;
this.format = format;
}
@Override
- public PCollection<BeamSqlRow> expand(PCollection<KV<byte[], byte[]>> input) {
- return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamSqlRow>() {
+ public PCollection<BeamRecord> expand(PCollection<KV<byte[], byte[]>> input) {
+ return input.apply("decodeRecord", ParDo.of(new DoFn<KV<byte[], byte[]>, BeamRecord>() {
@ProcessElement
public void processElement(ProcessContext c) {
String rowInString = new String(c.element().getValue());
@@ -83,24 +83,24 @@ public class BeamKafkaCSVTable extends BeamKafkaTable {
}
/**
- * A PTransform to convert {@link BeamSqlRow} to {@code KV<byte[], byte[]>}.
+ * A PTransform to convert {@link BeamRecord} to {@code KV<byte[], byte[]>}.
*
*/
public static class CsvRecorderEncoder
- extends PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>> {
- private BeamSqlRowType rowType;
+ extends PTransform<PCollection<BeamRecord>, PCollection<KV<byte[], byte[]>>> {
+ private BeamSqlRecordType rowType;
private CSVFormat format;
- public CsvRecorderEncoder(BeamSqlRowType rowType, CSVFormat format) {
+ public CsvRecorderEncoder(BeamSqlRecordType rowType, CSVFormat format) {
this.rowType = rowType;
this.format = format;
}
@Override
- public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamSqlRow> input) {
- return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, KV<byte[], byte[]>>() {
+ public PCollection<KV<byte[], byte[]>> expand(PCollection<BeamRecord> input) {
+ return input.apply("encodeRecord", ParDo.of(new DoFn<BeamRecord, KV<byte[], byte[]>>() {
@ProcessElement
public void processElement(ProcessContext c) {
- BeamSqlRow in = c.element();
+ BeamRecord in = c.element();
c.output(KV.of(new byte[] {}, BeamTableUtils.beamSqlRow2CsvLine(in, format).getBytes()));
}
}));
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
index 2cc664f..fac57bf 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/kafka/BeamKafkaTable.java
@@ -26,10 +26,10 @@ import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.io.kafka.KafkaIO;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
@@ -48,11 +48,11 @@ public abstract class BeamKafkaTable extends BaseBeamTable implements Serializab
private List<String> topics;
private Map<String, Object> configUpdates;
- protected BeamKafkaTable(BeamSqlRowType beamSqlRowType) {
+ protected BeamKafkaTable(BeamSqlRecordType beamSqlRowType) {
super(beamSqlRowType);
}
- public BeamKafkaTable(BeamSqlRowType beamSqlRowType, String bootstrapServers,
+ public BeamKafkaTable(BeamSqlRecordType beamSqlRowType, String bootstrapServers,
List<String> topics) {
super(beamSqlRowType);
this.bootstrapServers = bootstrapServers;
@@ -69,14 +69,14 @@ public abstract class BeamKafkaTable extends BaseBeamTable implements Serializab
return BeamIOType.UNBOUNDED;
}
- public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamSqlRow>>
+ public abstract PTransform<PCollection<KV<byte[], byte[]>>, PCollection<BeamRecord>>
getPTransformForInput();
- public abstract PTransform<PCollection<BeamSqlRow>, PCollection<KV<byte[], byte[]>>>
+ public abstract PTransform<PCollection<BeamRecord>, PCollection<KV<byte[], byte[]>>>
getPTransformForOutput();
@Override
- public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+ public PCollection<BeamRecord> buildIOReader(Pipeline pipeline) {
return PBegin.in(pipeline).apply("read",
KafkaIO.<byte[], byte[]>read()
.withBootstrapServers(bootstrapServers)
@@ -89,13 +89,13 @@ public abstract class BeamKafkaTable extends BaseBeamTable implements Serializab
}
@Override
- public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+ public PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter() {
checkArgument(topics != null && topics.size() == 1,
"Only one topic can be acceptable as output.");
- return new PTransform<PCollection<BeamSqlRow>, PDone>() {
+ return new PTransform<PCollection<BeamRecord>, PDone>() {
@Override
- public PDone expand(PCollection<BeamSqlRow> input) {
+ public PDone expand(PCollection<BeamRecord> input) {
return input.apply("out_reformat", getPTransformForOutput()).apply("persistent",
KafkaIO.<byte[], byte[]>write()
.withBootstrapServers(bootstrapServers)
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
index c44faab..0ec418c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTable.java
@@ -19,10 +19,10 @@
package org.apache.beam.sdk.extensions.sql.schema.text;
import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
@@ -46,25 +46,25 @@ public class BeamTextCSVTable extends BeamTextTable {
/**
* CSV table with {@link CSVFormat#DEFAULT DEFAULT} format.
*/
- public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern) {
+ public BeamTextCSVTable(BeamSqlRecordType beamSqlRowType, String filePattern) {
this(beamSqlRowType, filePattern, CSVFormat.DEFAULT);
}
- public BeamTextCSVTable(BeamSqlRowType beamSqlRowType, String filePattern,
+ public BeamTextCSVTable(BeamSqlRecordType beamSqlRowType, String filePattern,
CSVFormat csvFormat) {
super(beamSqlRowType, filePattern);
this.csvFormat = csvFormat;
}
@Override
- public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+ public PCollection<BeamRecord> buildIOReader(Pipeline pipeline) {
return PBegin.in(pipeline).apply("decodeRecord", TextIO.read().from(filePattern))
.apply("parseCSVLine",
new BeamTextCSVTableIOReader(beamSqlRowType, filePattern, csvFormat));
}
@Override
- public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+ public PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter() {
return new BeamTextCSVTableIOWriter(beamSqlRowType, filePattern, csvFormat);
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
index 06109c3..ecb77e0 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOReader.java
@@ -19,12 +19,12 @@
package org.apache.beam.sdk.extensions.sql.schema.text;
import java.io.Serializable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.commons.csv.CSVFormat;
@@ -32,13 +32,13 @@ import org.apache.commons.csv.CSVFormat;
* IOReader for {@code BeamTextCSVTable}.
*/
public class BeamTextCSVTableIOReader
- extends PTransform<PCollection<String>, PCollection<BeamSqlRow>>
+ extends PTransform<PCollection<String>, PCollection<BeamRecord>>
implements Serializable {
private String filePattern;
- protected BeamSqlRowType beamSqlRowType;
+ protected BeamSqlRecordType beamSqlRowType;
protected CSVFormat csvFormat;
- public BeamTextCSVTableIOReader(BeamSqlRowType beamSqlRowType, String filePattern,
+ public BeamTextCSVTableIOReader(BeamSqlRecordType beamSqlRowType, String filePattern,
CSVFormat csvFormat) {
this.filePattern = filePattern;
this.beamSqlRowType = beamSqlRowType;
@@ -46,8 +46,8 @@ public class BeamTextCSVTableIOReader
}
@Override
- public PCollection<BeamSqlRow> expand(PCollection<String> input) {
- return input.apply(ParDo.of(new DoFn<String, BeamSqlRow>() {
+ public PCollection<BeamRecord> expand(PCollection<String> input) {
+ return input.apply(ParDo.of(new DoFn<String, BeamRecord>() {
@ProcessElement
public void processElement(ProcessContext ctx) {
String str = ctx.element();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
index 1684b37..c616973 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextCSVTableIOWriter.java
@@ -19,13 +19,13 @@
package org.apache.beam.sdk.extensions.sql.schema.text;
import java.io.Serializable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
import org.apache.commons.csv.CSVFormat;
@@ -33,24 +33,24 @@ import org.apache.commons.csv.CSVFormat;
/**
* IOWriter for {@code BeamTextCSVTable}.
*/
-public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamSqlRow>, PDone>
+public class BeamTextCSVTableIOWriter extends PTransform<PCollection<BeamRecord>, PDone>
implements Serializable {
private String filePattern;
- protected BeamSqlRowType beamSqlRowType;
+ protected BeamSqlRecordType beamSqlRowType;
protected CSVFormat csvFormat;
- public BeamTextCSVTableIOWriter(BeamSqlRowType beamSqlRowType, String filePattern,
+ public BeamTextCSVTableIOWriter(BeamSqlRecordType beamSqlRowType, String filePattern,
CSVFormat csvFormat) {
this.filePattern = filePattern;
this.beamSqlRowType = beamSqlRowType;
this.csvFormat = csvFormat;
}
- @Override public PDone expand(PCollection<BeamSqlRow> input) {
- return input.apply("encodeRecord", ParDo.of(new DoFn<BeamSqlRow, String>() {
+ @Override public PDone expand(PCollection<BeamRecord> input) {
+ return input.apply("encodeRecord", ParDo.of(new DoFn<BeamRecord, String>() {
@ProcessElement public void processElement(ProcessContext ctx) {
- BeamSqlRow row = ctx.element();
+ BeamRecord row = ctx.element();
ctx.output(BeamTableUtils.beamSqlRow2CsvLine(row, csvFormat));
}
})).apply(TextIO.write().to(filePattern));
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
index e85608d..4284366 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/text/BeamTextTable.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.schema.text;
import java.io.Serializable;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
import org.apache.beam.sdk.extensions.sql.schema.BeamIOType;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
/**
* {@code BeamTextTable} represents a text file/directory(backed by {@code TextIO}).
@@ -29,7 +29,7 @@ import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
public abstract class BeamTextTable extends BaseBeamTable implements Serializable {
protected String filePattern;
- protected BeamTextTable(BeamSqlRowType beamSqlRowType, String filePattern) {
+ protected BeamTextTable(BeamSqlRecordType beamSqlRowType, String filePattern) {
super(beamSqlRowType);
this.filePattern = filePattern;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
index e6ca18f..8501157 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
@@ -19,9 +19,9 @@ package org.apache.beam.sdk.extensions.sql;
import java.sql.Types;
import java.util.Arrays;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -49,16 +49,16 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runAggregationWithoutWindow(unboundedInput1);
}
- private void runAggregationWithoutWindow(PCollection<BeamSqlRow> input) throws Exception {
+ private void runAggregationWithoutWindow(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int2, COUNT(*) AS `size` FROM PCOLLECTION GROUP BY f_int2";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
input.apply("testAggregationWithoutWindow", BeamSql.simpleQuery(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int2", "size"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int2", "size"),
Arrays.asList(Types.INTEGER, Types.BIGINT));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("f_int2", 0);
record.addField("size", 4L);
@@ -83,7 +83,7 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runAggregationFunctions(unboundedInput1);
}
- private void runAggregationFunctions(PCollection<BeamSqlRow> input) throws Exception{
+ private void runAggregationFunctions(PCollection<BeamRecord> input) throws Exception{
String sql = "select f_int2, count(*) as size, "
+ "sum(f_long) as sum1, avg(f_long) as avg1, max(f_long) as max1, min(f_long) as min1,"
+ "sum(f_short) as sum2, avg(f_short) as avg2, max(f_short) as max2, min(f_short) as min2,"
@@ -94,11 +94,11 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
+ "max(f_timestamp) as max6, min(f_timestamp) as min6 "
+ "FROM TABLE_A group by f_int2";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testAggregationFunctions", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(
Arrays.asList("f_int2", "size", "sum1", "avg1", "max1", "min1", "sum2", "avg2", "max2",
"min2", "sum3", "avg3", "max3", "min3", "sum4", "avg4", "max4", "min4", "sum5", "avg5",
"max5", "min5", "max6", "min6"),
@@ -108,7 +108,7 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
Types.FLOAT, Types.FLOAT, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE,
Types.TIMESTAMP, Types.TIMESTAMP));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("f_int2", 0);
record.addField("size", 4L);
@@ -161,28 +161,28 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runDistinct(unboundedInput1);
}
- private void runDistinct(PCollection<BeamSqlRow> input) throws Exception {
+ private void runDistinct(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT distinct f_int, f_long FROM PCOLLECTION ";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
input.apply("testDistinct", BeamSql.simpleQuery(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int", "f_long"),
Arrays.asList(Types.INTEGER, Types.BIGINT));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int", 1);
record1.addField("f_long", 1000L);
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int", 2);
record2.addField("f_long", 2000L);
- BeamSqlRow record3 = new BeamSqlRow(resultType);
+ BeamRecord record3 = new BeamRecord(resultType);
record3.addField("f_int", 3);
record3.addField("f_long", 3000L);
- BeamSqlRow record4 = new BeamSqlRow(resultType);
+ BeamRecord record4 = new BeamRecord(resultType);
record4.addField("f_int", 4);
record4.addField("f_long", 4000L);
@@ -207,27 +207,27 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runTumbleWindow(unboundedInput1);
}
- private void runTumbleWindow(PCollection<BeamSqlRow> input) throws Exception {
+ private void runTumbleWindow(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+ " TUMBLE_START(f_timestamp, INTERVAL '1' HOUR) AS `window_start`"
+ " FROM TABLE_A"
+ " GROUP BY f_int2, TUMBLE(f_timestamp, INTERVAL '1' HOUR)";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testTumbleWindow", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(
Arrays.asList("f_int2", "size", "window_start"),
Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int2", 0);
record1.addField("size", 3L);
record1.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int2", 0);
record2.addField("size", 1L);
record2.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
@@ -255,40 +255,40 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runHopWindow(unboundedInput1);
}
- private void runHopWindow(PCollection<BeamSqlRow> input) throws Exception {
+ private void runHopWindow(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+ " HOP_START(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE) AS `window_start`"
+ " FROM PCOLLECTION"
+ " GROUP BY f_int2, HOP(f_timestamp, INTERVAL '1' HOUR, INTERVAL '30' MINUTE)";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
input.apply("testHopWindow", BeamSql.simpleQuery(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(
Arrays.asList("f_int2", "size", "window_start"),
Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int2", 0);
record1.addField("size", 3L);
record1.addField("window_start", FORMAT.parse("2017-01-01 00:30:00"));
record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 00:30:00").getTime()));
record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int2", 0);
record2.addField("size", 3L);
record2.addField("window_start", FORMAT.parse("2017-01-01 01:00:00"));
record2.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:00:00").getTime()));
record2.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:00:00").getTime()));
- BeamSqlRow record3 = new BeamSqlRow(resultType);
+ BeamRecord record3 = new BeamRecord(resultType);
record3.addField("f_int2", 0);
record3.addField("size", 1L);
record3.addField("window_start", FORMAT.parse("2017-01-01 01:30:00"));
record3.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:30:00").getTime()));
record3.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 02:30:00").getTime()));
- BeamSqlRow record4 = new BeamSqlRow(resultType);
+ BeamRecord record4 = new BeamRecord(resultType);
record4.addField("f_int2", 0);
record4.addField("size", 1L);
record4.addField("window_start", FORMAT.parse("2017-01-01 02:00:00"));
@@ -316,27 +316,27 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
runSessionWindow(unboundedInput1);
}
- private void runSessionWindow(PCollection<BeamSqlRow> input) throws Exception {
+ private void runSessionWindow(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int2, COUNT(*) AS `size`,"
+ " SESSION_START(f_timestamp, INTERVAL '5' MINUTE) AS `window_start`"
+ " FROM TABLE_A"
+ " GROUP BY f_int2, SESSION(f_timestamp, INTERVAL '5' MINUTE)";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testSessionWindow", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(
Arrays.asList("f_int2", "size", "window_start"),
Arrays.asList(Types.INTEGER, Types.BIGINT, Types.TIMESTAMP));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int2", 0);
record1.addField("size", 3L);
record1.addField("window_start", FORMAT.parse("2017-01-01 01:01:03"));
record1.setWindowStart(new Instant(FORMAT.parse("2017-01-01 01:01:03").getTime()));
record1.setWindowEnd(new Instant(FORMAT.parse("2017-01-01 01:11:03").getTime()));
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int2", 0);
record2.addField("size", 1L);
record2.addField("window_start", FORMAT.parse("2017-01-01 02:04:03"));
@@ -357,8 +357,8 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
String sql = "SELECT f_int2, COUNT(*) AS `size` FROM TABLE_A "
+ "GROUP BY f_int2, TUMBLE(f_long, INTERVAL '1' HOUR)";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), boundedInput1)
.apply("testWindowOnNonTimestampField", BeamSql.query(sql));
pipeline.run().waitUntilFinish();
@@ -372,7 +372,7 @@ public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
String sql = "SELECT f_int2, COUNT(DISTINCT *) AS `size` FROM PCOLLECTION GROUP BY f_int2";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
boundedInput1.apply("testUnsupportedDistinct", BeamSql.simpleQuery(sql));
pipeline.run().waitUntilFinish();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
index 0c1ce1c..d09caf0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
@@ -25,12 +25,11 @@ import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestStream;
import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.joda.time.Instant;
@@ -53,20 +52,20 @@ public class BeamSqlDslBase {
@Rule
public ExpectedException exceptions = ExpectedException.none();
- public static BeamSqlRowType rowTypeInTableA;
- public static List<BeamSqlRow> recordsInTableA;
+ public static BeamSqlRecordType rowTypeInTableA;
+ public static List<BeamRecord> recordsInTableA;
//bounded PCollections
- public PCollection<BeamSqlRow> boundedInput1;
- public PCollection<BeamSqlRow> boundedInput2;
+ public PCollection<BeamRecord> boundedInput1;
+ public PCollection<BeamRecord> boundedInput2;
//unbounded PCollections
- public PCollection<BeamSqlRow> unboundedInput1;
- public PCollection<BeamSqlRow> unboundedInput2;
+ public PCollection<BeamRecord> unboundedInput1;
+ public PCollection<BeamRecord> unboundedInput2;
@BeforeClass
public static void prepareClass() throws ParseException {
- rowTypeInTableA = BeamSqlRowType.create(
+ rowTypeInTableA = BeamSqlRecordType.create(
Arrays.asList("f_int", "f_long", "f_short", "f_byte", "f_float", "f_double", "f_string",
"f_timestamp", "f_int2", "f_decimal"),
Arrays.asList(Types.INTEGER, Types.BIGINT, Types.SMALLINT, Types.TINYINT, Types.FLOAT,
@@ -78,20 +77,20 @@ public class BeamSqlDslBase {
@Before
public void preparePCollections(){
boundedInput1 = PBegin.in(pipeline).apply("boundedInput1",
- Create.of(recordsInTableA).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+ Create.of(recordsInTableA).withCoder(rowTypeInTableA.getRecordCoder()));
boundedInput2 = PBegin.in(pipeline).apply("boundedInput2",
- Create.of(recordsInTableA.get(0)).withCoder(new BeamSqlRowCoder(rowTypeInTableA)));
+ Create.of(recordsInTableA.get(0)).withCoder(rowTypeInTableA.getRecordCoder()));
unboundedInput1 = prepareUnboundedPCollection1();
unboundedInput2 = prepareUnboundedPCollection2();
}
- private PCollection<BeamSqlRow> prepareUnboundedPCollection1() {
- TestStream.Builder<BeamSqlRow> values = TestStream
- .create(new BeamSqlRowCoder(rowTypeInTableA));
+ private PCollection<BeamRecord> prepareUnboundedPCollection1() {
+ TestStream.Builder<BeamRecord> values = TestStream
+ .create(rowTypeInTableA.getRecordCoder());
- for (BeamSqlRow row : recordsInTableA) {
+ for (BeamRecord row : recordsInTableA) {
values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
values = values.addElements(row);
}
@@ -99,21 +98,21 @@ public class BeamSqlDslBase {
return PBegin.in(pipeline).apply("unboundedInput1", values.advanceWatermarkToInfinity());
}
- private PCollection<BeamSqlRow> prepareUnboundedPCollection2() {
- TestStream.Builder<BeamSqlRow> values = TestStream
- .create(new BeamSqlRowCoder(rowTypeInTableA));
+ private PCollection<BeamRecord> prepareUnboundedPCollection2() {
+ TestStream.Builder<BeamRecord> values = TestStream
+ .create(rowTypeInTableA.getRecordCoder());
- BeamSqlRow row = recordsInTableA.get(0);
+ BeamRecord row = recordsInTableA.get(0);
values = values.advanceWatermarkTo(new Instant(row.getDate("f_timestamp")));
values = values.addElements(row);
return PBegin.in(pipeline).apply("unboundedInput2", values.advanceWatermarkToInfinity());
}
- private static List<BeamSqlRow> prepareInputRowsInTableA() throws ParseException{
- List<BeamSqlRow> rows = new ArrayList<>();
+ private static List<BeamRecord> prepareInputRowsInTableA() throws ParseException{
+ List<BeamRecord> rows = new ArrayList<>();
- BeamSqlRow row1 = new BeamSqlRow(rowTypeInTableA);
+ BeamRecord row1 = new BeamRecord(rowTypeInTableA);
row1.addField(0, 1);
row1.addField(1, 1000L);
row1.addField(2, Short.valueOf("1"));
@@ -126,7 +125,7 @@ public class BeamSqlDslBase {
row1.addField(9, new BigDecimal(1));
rows.add(row1);
- BeamSqlRow row2 = new BeamSqlRow(rowTypeInTableA);
+ BeamRecord row2 = new BeamRecord(rowTypeInTableA);
row2.addField(0, 2);
row2.addField(1, 2000L);
row2.addField(2, Short.valueOf("2"));
@@ -139,7 +138,7 @@ public class BeamSqlDslBase {
row2.addField(9, new BigDecimal(2));
rows.add(row2);
- BeamSqlRow row3 = new BeamSqlRow(rowTypeInTableA);
+ BeamRecord row3 = new BeamRecord(rowTypeInTableA);
row3.addField(0, 3);
row3.addField(1, 3000L);
row3.addField(2, Short.valueOf("3"));
@@ -152,7 +151,7 @@ public class BeamSqlDslBase {
row3.addField(9, new BigDecimal(3));
rows.add(row3);
- BeamSqlRow row4 = new BeamSqlRow(rowTypeInTableA);
+ BeamRecord row4 = new BeamRecord(rowTypeInTableA);
row4.addField(0, 4);
row4.addField(1, 4000L);
row4.addField(2, Short.valueOf("4"));
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
index 16b6426..e1d463b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslFilterTest.java
@@ -17,8 +17,8 @@
*/
package org.apache.beam.sdk.extensions.sql;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -44,10 +44,10 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
runSingleFilter(unboundedInput1);
}
- private void runSingleFilter(PCollection<BeamSqlRow> input) throws Exception {
+ private void runSingleFilter(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT * FROM PCOLLECTION WHERE f_int = 1";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
input.apply("testSingleFilter", BeamSql.simpleQuery(sql));
PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
@@ -71,12 +71,12 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
runCompositeFilter(unboundedInput1);
}
- private void runCompositeFilter(PCollection<BeamSqlRow> input) throws Exception {
+ private void runCompositeFilter(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT * FROM TABLE_A"
+ " WHERE f_int > 1 AND (f_long < 3000 OR f_string = 'string_row3')";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testCompositeFilter", BeamSql.query(sql));
PAssert.that(result).containsInAnyOrder(recordsInTableA.get(1), recordsInTableA.get(2));
@@ -100,11 +100,11 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
runNoReturnFilter(unboundedInput1);
}
- private void runNoReturnFilter(PCollection<BeamSqlRow> input) throws Exception {
+ private void runNoReturnFilter(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT * FROM TABLE_A WHERE f_int < 1";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testNoReturnFilter", BeamSql.query(sql));
PAssert.that(result).empty();
@@ -120,8 +120,8 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
String sql = "SELECT * FROM TABLE_B WHERE f_int < 1";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), boundedInput1)
.apply("testFromInvalidTableName1", BeamSql.query(sql));
pipeline.run().waitUntilFinish();
@@ -135,7 +135,7 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
String sql = "SELECT * FROM PCOLLECTION_NA";
- PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+ PCollection<BeamRecord> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
pipeline.run().waitUntilFinish();
}
@@ -148,7 +148,7 @@ public class BeamSqlDslFilterTest extends BeamSqlDslBase {
String sql = "SELECT * FROM PCOLLECTION WHERE f_int_na = 0";
- PCollection<BeamSqlRow> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
+ PCollection<BeamRecord> result = boundedInput1.apply(BeamSql.simpleQuery(sql));
pipeline.run().waitUntilFinish();
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
index d75af9b..d5d0a24 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
@@ -23,11 +23,11 @@ import static org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRelBoundedVsBo
import java.sql.Types;
import java.util.Arrays;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -41,8 +41,8 @@ public class BeamSqlDslJoinTest {
@Rule
public final TestPipeline pipeline = TestPipeline.create();
- private static final BeamSqlRowType SOURCE_RECORD_TYPE =
- BeamSqlRowType.create(
+ private static final BeamSqlRecordType SOURCE_RECORD_TYPE =
+ BeamSqlRecordType.create(
Arrays.asList(
"order_id", "site_id", "price"
),
@@ -51,11 +51,10 @@ public class BeamSqlDslJoinTest {
)
);
- private static final BeamSqlRowCoder SOURCE_CODER =
- new BeamSqlRowCoder(SOURCE_RECORD_TYPE);
+ private static final BeamRecordCoder SOURCE_CODER = SOURCE_RECORD_TYPE.getRecordCoder();
- private static final BeamSqlRowType RESULT_RECORD_TYPE =
- BeamSqlRowType.create(
+ private static final BeamSqlRecordType RESULT_RECORD_TYPE =
+ BeamSqlRecordType.create(
Arrays.asList(
"order_id", "site_id", "price", "order_id0", "site_id0", "price0"
),
@@ -65,8 +64,7 @@ public class BeamSqlDslJoinTest {
)
);
- private static final BeamSqlRowCoder RESULT_CODER =
- new BeamSqlRowCoder(RESULT_RECORD_TYPE);
+ private static final BeamRecordCoder RESULT_CODER = RESULT_RECORD_TYPE.getRecordCoder();
@Test
public void testInnerJoin() throws Exception {
@@ -178,13 +176,13 @@ public class BeamSqlDslJoinTest {
pipeline.run();
}
- private PCollection<BeamSqlRow> queryFromOrderTables(String sql) {
+ private PCollection<BeamRecord> queryFromOrderTables(String sql) {
return PCollectionTuple
.of(
- new TupleTag<BeamSqlRow>("ORDER_DETAILS1"),
+ new TupleTag<BeamRecord>("ORDER_DETAILS1"),
ORDER_DETAILS1.buildIOReader(pipeline).setCoder(SOURCE_CODER)
)
- .and(new TupleTag<BeamSqlRow>("ORDER_DETAILS2"),
+ .and(new TupleTag<BeamRecord>("ORDER_DETAILS2"),
ORDER_DETAILS2.buildIOReader(pipeline).setCoder(SOURCE_CODER)
).apply("join", BeamSql.query(sql)).setCoder(RESULT_CODER);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
index 6468011..ddb90d5 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
@@ -19,9 +19,9 @@ package org.apache.beam.sdk.extensions.sql;
import java.sql.Types;
import java.util.Arrays;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -47,10 +47,10 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
runSelectAll(unboundedInput2);
}
- private void runSelectAll(PCollection<BeamSqlRow> input) throws Exception {
+ private void runSelectAll(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT * FROM PCOLLECTION";
- PCollection<BeamSqlRow> result =
+ PCollection<BeamRecord> result =
input.apply("testSelectAll", BeamSql.simpleQuery(sql));
PAssert.that(result).containsInAnyOrder(recordsInTableA.get(0));
@@ -74,17 +74,17 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
runPartialFields(unboundedInput2);
}
- private void runPartialFields(PCollection<BeamSqlRow> input) throws Exception {
+ private void runPartialFields(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int, f_long FROM TABLE_A";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testPartialFields", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int", "f_long"),
Arrays.asList(Types.INTEGER, Types.BIGINT));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
record.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
@@ -109,29 +109,29 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
runPartialFieldsInMultipleRow(unboundedInput1);
}
- private void runPartialFieldsInMultipleRow(PCollection<BeamSqlRow> input) throws Exception {
+ private void runPartialFieldsInMultipleRow(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int, f_long FROM TABLE_A";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testPartialFieldsInMultipleRow", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int", "f_long"),
Arrays.asList(Types.INTEGER, Types.BIGINT));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
- BeamSqlRow record3 = new BeamSqlRow(resultType);
+ BeamRecord record3 = new BeamRecord(resultType);
record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
- BeamSqlRow record4 = new BeamSqlRow(resultType);
+ BeamRecord record4 = new BeamRecord(resultType);
record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
@@ -156,29 +156,29 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
runPartialFieldsInRows(unboundedInput1);
}
- private void runPartialFieldsInRows(PCollection<BeamSqlRow> input) throws Exception {
+ private void runPartialFieldsInRows(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT f_int, f_long FROM TABLE_A";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testPartialFieldsInRows", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("f_int", "f_long"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("f_int", "f_long"),
Arrays.asList(Types.INTEGER, Types.BIGINT));
- BeamSqlRow record1 = new BeamSqlRow(resultType);
+ BeamRecord record1 = new BeamRecord(resultType);
record1.addField("f_int", recordsInTableA.get(0).getFieldValue(0));
record1.addField("f_long", recordsInTableA.get(0).getFieldValue(1));
- BeamSqlRow record2 = new BeamSqlRow(resultType);
+ BeamRecord record2 = new BeamRecord(resultType);
record2.addField("f_int", recordsInTableA.get(1).getFieldValue(0));
record2.addField("f_long", recordsInTableA.get(1).getFieldValue(1));
- BeamSqlRow record3 = new BeamSqlRow(resultType);
+ BeamRecord record3 = new BeamRecord(resultType);
record3.addField("f_int", recordsInTableA.get(2).getFieldValue(0));
record3.addField("f_long", recordsInTableA.get(2).getFieldValue(1));
- BeamSqlRow record4 = new BeamSqlRow(resultType);
+ BeamRecord record4 = new BeamRecord(resultType);
record4.addField("f_int", recordsInTableA.get(3).getFieldValue(0));
record4.addField("f_long", recordsInTableA.get(3).getFieldValue(1));
@@ -203,17 +203,17 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
runLiteralField(unboundedInput2);
}
- public void runLiteralField(PCollection<BeamSqlRow> input) throws Exception {
+ public void runLiteralField(PCollection<BeamRecord> input) throws Exception {
String sql = "SELECT 1 as literal_field FROM TABLE_A";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), input)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), input)
.apply("testLiteralField", BeamSql.query(sql));
- BeamSqlRowType resultType = BeamSqlRowType.create(Arrays.asList("literal_field"),
+ BeamSqlRecordType resultType = BeamSqlRecordType.create(Arrays.asList("literal_field"),
Arrays.asList(Types.INTEGER));
- BeamSqlRow record = new BeamSqlRow(resultType);
+ BeamRecord record = new BeamRecord(resultType);
record.addField("literal_field", 1);
PAssert.that(result).containsInAnyOrder(record);
@@ -229,8 +229,8 @@ public class BeamSqlDslProjectTest extends BeamSqlDslBase {
String sql = "SELECT f_int_na FROM TABLE_A";
- PCollection<BeamSqlRow> result =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("TABLE_A"), boundedInput1)
+ PCollection<BeamRecord> result =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("TABLE_A"), boundedInput1)
.apply("testProjectUnknownField", BeamSql.query(sql));
pipeline.run().waitUntilFinish();
[5/7] beam git commit: refactor BeamRecord, BeamRecordType,
BeamSqlRecordType, BeamRecordCoder
Posted by ro...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
index ea5f749..06dce91 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlPositionExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -56,7 +56,7 @@ public class BeamSqlPositionExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String targetStr = opValueEvaluated(0, inputRow);
String containingStr = opValueEvaluated(1, inputRow);
int from = -1;
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
index 25f205a..f8582aa 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlSubstringExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -54,7 +54,7 @@ public class BeamSqlSubstringExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
int idx = opValueEvaluated(1, inputRow);
int startIdx = idx;
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
index 9493e24..9c2a7ae 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlTrimExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.fun.SqlTrimFunction;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -58,7 +58,7 @@ public class BeamSqlTrimExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
if (operands.size() == 1) {
return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
opValueEvaluated(0, inputRow).toString().trim());
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
index 9769c0e..94ac2e2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlUpperExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlUpperExpression extends BeamSqlStringUnaryExpression {
super(operands, SqlTypeName.VARCHAR);
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toUpperCase());
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
index dd01a87..b421bc3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamQueryPlanner.java
@@ -27,7 +27,7 @@ import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.adapter.java.JavaTypeFactory;
@@ -107,7 +107,7 @@ public class BeamQueryPlanner {
* which is linked with the given {@code pipeline}. The final output stream is returned as
* {@code PCollection} so more operations can be applied.
*/
- public PCollection<BeamSqlRow> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
+ public PCollection<BeamRecord> compileBeamPipeline(String sqlStatement, Pipeline basePipeline
, BeamSqlEnv sqlEnv) throws Exception {
BeamRelNode relNode = convertToBeamRel(sqlStatement);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
index 8e78684..d91b484 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
@@ -19,13 +19,12 @@ package org.apache.beam.sdk.extensions.sql.impl.rel;
import java.util.ArrayList;
import java.util.List;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.WithKeys;
@@ -34,6 +33,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.Trigger;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
@@ -55,7 +55,7 @@ import org.joda.time.Duration;
*/
public class BeamAggregationRel extends Aggregate implements BeamRelNode {
private int windowFieldIdx = -1;
- private WindowFn<BeamSqlRow, BoundedWindow> windowFn;
+ private WindowFn<BeamRecord, BoundedWindow> windowFn;
private Trigger trigger;
private Duration allowedLatence = Duration.ZERO;
@@ -71,12 +71,12 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
}
@Override
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
RelNode input = getInput();
String stageName = BeamSqlRelUtils.getStageName(this) + "_";
- PCollection<BeamSqlRow> upstream =
+ PCollection<BeamRecord> upstream =
BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
if (windowFieldIdx != -1) {
upstream = upstream.apply(stageName + "assignEventTimestamp", WithTimestamps
@@ -84,14 +84,14 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
.setCoder(upstream.getCoder());
}
- PCollection<BeamSqlRow> windowStream = upstream.apply(stageName + "window",
+ PCollection<BeamRecord> windowStream = upstream.apply(stageName + "window",
Window.into(windowFn)
.triggering(trigger)
.withAllowedLateness(allowedLatence)
.accumulatingFiredPanes());
- BeamSqlRowCoder keyCoder = new BeamSqlRowCoder(exKeyFieldsSchema(input.getRowType()));
- PCollection<KV<BeamSqlRow, BeamSqlRow>> exCombineByStream = windowStream.apply(
+ BeamRecordCoder keyCoder = exKeyFieldsSchema(input.getRowType()).getRecordCoder();
+ PCollection<KV<BeamRecord, BeamRecord>> exCombineByStream = windowStream.apply(
stageName + "exCombineBy",
WithKeys
.of(new BeamAggregationTransforms.AggregationGroupByKeyFn(
@@ -99,19 +99,19 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
.setCoder(KvCoder.of(keyCoder, upstream.getCoder()));
- BeamSqlRowCoder aggCoder = new BeamSqlRowCoder(exAggFieldsSchema());
+ BeamRecordCoder aggCoder = exAggFieldsSchema().getRecordCoder();
- PCollection<KV<BeamSqlRow, BeamSqlRow>> aggregatedStream = exCombineByStream.apply(
+ PCollection<KV<BeamRecord, BeamRecord>> aggregatedStream = exCombineByStream.apply(
stageName + "combineBy",
- Combine.<BeamSqlRow, BeamSqlRow, BeamSqlRow>perKey(
+ Combine.<BeamRecord, BeamRecord, BeamRecord>perKey(
new BeamAggregationTransforms.AggregationAdaptor(getAggCallList(),
CalciteUtils.toBeamRowType(input.getRowType()))))
.setCoder(KvCoder.of(keyCoder, aggCoder));
- PCollection<BeamSqlRow> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
+ PCollection<BeamRecord> mergedStream = aggregatedStream.apply(stageName + "mergeRecord",
ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(
CalciteUtils.toBeamRowType(getRowType()), getAggCallList(), windowFieldIdx)));
- mergedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ mergedStream.setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return mergedStream;
}
@@ -119,8 +119,8 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
/**
* Type of sub-rowrecord used as Group-By keys.
*/
- private BeamSqlRowType exKeyFieldsSchema(RelDataType relDataType) {
- BeamSqlRowType inputRowType = CalciteUtils.toBeamRowType(relDataType);
+ private BeamSqlRecordType exKeyFieldsSchema(RelDataType relDataType) {
+ BeamSqlRecordType inputRowType = CalciteUtils.toBeamRowType(relDataType);
List<String> fieldNames = new ArrayList<>();
List<Integer> fieldTypes = new ArrayList<>();
for (int i : groupSet.asList()) {
@@ -129,13 +129,13 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
fieldTypes.add(inputRowType.getFieldsType().get(i));
}
}
- return BeamSqlRowType.create(fieldNames, fieldTypes);
+ return BeamSqlRecordType.create(fieldNames, fieldTypes);
}
/**
* Type of sub-rowrecord, that represents the list of aggregation fields.
*/
- private BeamSqlRowType exAggFieldsSchema() {
+ private BeamSqlRecordType exAggFieldsSchema() {
List<String> fieldNames = new ArrayList<>();
List<Integer> fieldTypes = new ArrayList<>();
for (AggregateCall ac : getAggCallList()) {
@@ -143,7 +143,7 @@ public class BeamAggregationRel extends Aggregate implements BeamRelNode {
fieldTypes.add(CalciteUtils.toJavaType(ac.type.getSqlTypeName()));
}
- return BeamSqlRowType.create(fieldNames, fieldTypes);
+ return BeamSqlRecordType.create(fieldNames, fieldTypes);
}
@Override
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
index b453db4..8fe5be4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamFilterRel.java
@@ -22,9 +22,8 @@ import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExec
import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutor;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlFilterFn;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -50,19 +49,19 @@ public class BeamFilterRel extends Filter implements BeamRelNode {
}
@Override
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
RelNode input = getInput();
String stageName = BeamSqlRelUtils.getStageName(this);
- PCollection<BeamSqlRow> upstream =
+ PCollection<BeamRecord> upstream =
BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
- PCollection<BeamSqlRow> filterStream = upstream.apply(stageName,
+ PCollection<BeamRecord> filterStream = upstream.apply(stageName,
ParDo.of(new BeamSqlFilterFn(getRelTypeName(), executor)));
- filterStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ filterStream.setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return filterStream;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
index d5eb210..1e3eb4c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
@@ -21,7 +21,7 @@ import com.google.common.base.Joiner;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -55,12 +55,12 @@ public class BeamIOSinkRel extends TableModify implements BeamRelNode {
* which is the persisted PCollection.
*/
@Override
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
RelNode input = getInput();
String stageName = BeamSqlRelUtils.getStageName(this);
- PCollection<BeamSqlRow> upstream =
+ PCollection<BeamRecord> upstream =
BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
index 5179eba..254f990 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
@@ -21,8 +21,7 @@ import com.google.common.base.Joiner;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -42,21 +41,21 @@ public class BeamIOSourceRel extends TableScan implements BeamRelNode {
}
@Override
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
- TupleTag<BeamSqlRow> sourceTupleTag = new TupleTag<>(sourceName);
+ TupleTag<BeamRecord> sourceTupleTag = new TupleTag<>(sourceName);
if (inputPCollections.has(sourceTupleTag)) {
//choose PCollection from input PCollectionTuple if exists there.
- PCollection<BeamSqlRow> sourceStream = inputPCollections
- .get(new TupleTag<BeamSqlRow>(sourceName));
+ PCollection<BeamRecord> sourceStream = inputPCollections
+ .get(new TupleTag<BeamRecord>(sourceName));
return sourceStream;
} else {
//If not, the source PColection is provided with BaseBeamTable.buildIOReader().
BaseBeamTable sourceTable = sqlEnv.findTable(sourceName);
return sourceTable.buildIOReader(inputPCollections.getPipeline())
- .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ .setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
index d6ab52d..5919329 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -51,7 +51,7 @@ public class BeamIntersectRel extends Intersect implements BeamRelNode {
return new BeamIntersectRel(getCluster(), traitSet, inputs, all);
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
index 2de2a89..9e5ce2f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
@@ -27,14 +27,13 @@ import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamJoinTransforms;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.windowing.IncompatibleWindowException;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
@@ -93,15 +92,15 @@ public class BeamJoinRel extends Join implements BeamRelNode {
joinType);
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections,
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections,
BeamSqlEnv sqlEnv)
throws Exception {
BeamRelNode leftRelNode = BeamSqlRelUtils.getBeamRelInput(left);
- BeamSqlRowType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
- PCollection<BeamSqlRow> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+ BeamSqlRecordType leftRowType = CalciteUtils.toBeamRowType(left.getRowType());
+ PCollection<BeamRecord> leftRows = leftRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
final BeamRelNode rightRelNode = BeamSqlRelUtils.getBeamRelInput(right);
- PCollection<BeamSqlRow> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
+ PCollection<BeamRecord> rightRows = rightRelNode.buildBeamPipeline(inputPCollections, sqlEnv);
String stageName = BeamSqlRelUtils.getStageName(this);
WindowFn leftWinFn = leftRows.getWindowingStrategy().getWindowFn();
@@ -119,24 +118,24 @@ public class BeamJoinRel extends Join implements BeamRelNode {
names.add("c" + i);
types.add(leftRowType.getFieldsType().get(pairs.get(i).getKey()));
}
- BeamSqlRowType extractKeyRowType = BeamSqlRowType.create(names, types);
+ BeamSqlRecordType extractKeyRowType = BeamSqlRecordType.create(names, types);
- Coder extractKeyRowCoder = new BeamSqlRowCoder(extractKeyRowType);
+ Coder extractKeyRowCoder = extractKeyRowType.getRecordCoder();
// BeamSqlRow -> KV<BeamSqlRow, BeamSqlRow>
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows = leftRows
+ PCollection<KV<BeamRecord, BeamRecord>> extractedLeftRows = leftRows
.apply(stageName + "_left_ExtractJoinFields",
MapElements.via(new BeamJoinTransforms.ExtractJoinFields(true, pairs)))
.setCoder(KvCoder.of(extractKeyRowCoder, leftRows.getCoder()));
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows = rightRows
+ PCollection<KV<BeamRecord, BeamRecord>> extractedRightRows = rightRows
.apply(stageName + "_right_ExtractJoinFields",
MapElements.via(new BeamJoinTransforms.ExtractJoinFields(false, pairs)))
.setCoder(KvCoder.of(extractKeyRowCoder, rightRows.getCoder()));
// prepare the NullRows
- BeamSqlRow leftNullRow = buildNullRow(leftRelNode);
- BeamSqlRow rightNullRow = buildNullRow(rightRelNode);
+ BeamRecord leftNullRow = buildNullRow(leftRelNode);
+ BeamRecord rightNullRow = buildNullRow(rightRelNode);
// a regular join
if ((leftRows.isBounded() == PCollection.IsBounded.BOUNDED
@@ -184,11 +183,11 @@ public class BeamJoinRel extends Join implements BeamRelNode {
}
}
- private PCollection<BeamSqlRow> standardJoin(
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
- BeamSqlRow leftNullRow, BeamSqlRow rightNullRow, String stageName) {
- PCollection<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>> joinedRows = null;
+ private PCollection<BeamRecord> standardJoin(
+ PCollection<KV<BeamRecord, BeamRecord>> extractedLeftRows,
+ PCollection<KV<BeamRecord, BeamRecord>> extractedRightRows,
+ BeamRecord leftNullRow, BeamRecord rightNullRow, String stageName) {
+ PCollection<KV<BeamRecord, KV<BeamRecord, BeamRecord>>> joinedRows = null;
switch (joinType) {
case LEFT:
joinedRows = org.apache.beam.sdk.extensions.joinlibrary.Join
@@ -210,53 +209,53 @@ public class BeamJoinRel extends Join implements BeamRelNode {
break;
}
- PCollection<BeamSqlRow> ret = joinedRows
+ PCollection<BeamRecord> ret = joinedRows
.apply(stageName + "_JoinParts2WholeRow",
MapElements.via(new BeamJoinTransforms.JoinParts2WholeRow()))
- .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ .setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return ret;
}
- public PCollection<BeamSqlRow> sideInputJoin(
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedLeftRows,
- PCollection<KV<BeamSqlRow, BeamSqlRow>> extractedRightRows,
- BeamSqlRow leftNullRow, BeamSqlRow rightNullRow) {
+ public PCollection<BeamRecord> sideInputJoin(
+ PCollection<KV<BeamRecord, BeamRecord>> extractedLeftRows,
+ PCollection<KV<BeamRecord, BeamRecord>> extractedRightRows,
+ BeamRecord leftNullRow, BeamRecord rightNullRow) {
// we always make the Unbounded table on the left to do the sideInput join
// (will convert the result accordingly before return)
boolean swapped = (extractedLeftRows.isBounded() == PCollection.IsBounded.BOUNDED);
JoinRelType realJoinType =
(swapped && joinType != JoinRelType.INNER) ? JoinRelType.LEFT : joinType;
- PCollection<KV<BeamSqlRow, BeamSqlRow>> realLeftRows =
+ PCollection<KV<BeamRecord, BeamRecord>> realLeftRows =
swapped ? extractedRightRows : extractedLeftRows;
- PCollection<KV<BeamSqlRow, BeamSqlRow>> realRightRows =
+ PCollection<KV<BeamRecord, BeamRecord>> realRightRows =
swapped ? extractedLeftRows : extractedRightRows;
- BeamSqlRow realRightNullRow = swapped ? leftNullRow : rightNullRow;
+ BeamRecord realRightNullRow = swapped ? leftNullRow : rightNullRow;
// swapped still need to pass down because, we need to swap the result back.
return sideInputJoinHelper(realJoinType, realLeftRows, realRightRows,
realRightNullRow, swapped);
}
- private PCollection<BeamSqlRow> sideInputJoinHelper(
+ private PCollection<BeamRecord> sideInputJoinHelper(
JoinRelType joinType,
- PCollection<KV<BeamSqlRow, BeamSqlRow>> leftRows,
- PCollection<KV<BeamSqlRow, BeamSqlRow>> rightRows,
- BeamSqlRow rightNullRow, boolean swapped) {
- final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> rowsView = rightRows
- .apply(View.<BeamSqlRow, BeamSqlRow>asMultimap());
+ PCollection<KV<BeamRecord, BeamRecord>> leftRows,
+ PCollection<KV<BeamRecord, BeamRecord>> rightRows,
+ BeamRecord rightNullRow, boolean swapped) {
+ final PCollectionView<Map<BeamRecord, Iterable<BeamRecord>>> rowsView = rightRows
+ .apply(View.<BeamRecord, BeamRecord>asMultimap());
- PCollection<BeamSqlRow> ret = leftRows
+ PCollection<BeamRecord> ret = leftRows
.apply(ParDo.of(new BeamJoinTransforms.SideInputJoinDoFn(
joinType, rightNullRow, rowsView, swapped)).withSideInputs(rowsView))
- .setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ .setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return ret;
}
- private BeamSqlRow buildNullRow(BeamRelNode relNode) {
- BeamSqlRowType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
- BeamSqlRow nullRow = new BeamSqlRow(leftType);
+ private BeamRecord buildNullRow(BeamRelNode relNode) {
+ BeamSqlRecordType leftType = CalciteUtils.toBeamRowType(relNode.getRowType());
+ BeamRecord nullRow = new BeamRecord(leftType);
for (int i = 0; i < leftType.size(); i++) {
nullRow.addField(i, null);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
index 0075d3a..b55252a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -49,7 +49,7 @@ public class BeamMinusRel extends Minus implements BeamRelNode {
return new BeamMinusRel(getCluster(), traitSet, inputs, all);
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
index 6ccb156..b1ff629 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamProjectRel.java
@@ -23,9 +23,8 @@ import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExec
import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutor;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSqlProjectFn;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -60,20 +59,20 @@ public class BeamProjectRel extends Project implements BeamRelNode {
}
@Override
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
RelNode input = getInput();
String stageName = BeamSqlRelUtils.getStageName(this);
- PCollection<BeamSqlRow> upstream =
+ PCollection<BeamRecord> upstream =
BeamSqlRelUtils.getBeamRelInput(input).buildBeamPipeline(inputPCollections, sqlEnv);
BeamSqlExpressionExecutor executor = new BeamSqlFnExecutor(this);
- PCollection<BeamSqlRow> projectStream = upstream.apply(stageName, ParDo
+ PCollection<BeamRecord> projectStream = upstream.apply(stageName, ParDo
.of(new BeamSqlProjectFn(getRelTypeName(), executor,
CalciteUtils.toBeamRowType(rowType))));
- projectStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ projectStream.setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return projectStream;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
index 8a51cc7..b8b4293 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
@@ -18,7 +18,7 @@
package org.apache.beam.sdk.extensions.sql.impl.rel;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.rel.RelNode;
@@ -33,6 +33,6 @@ public interface BeamRelNode extends RelNode {
* {@code BeamQueryPlanner} visits it with a DFS(Depth-First-Search)
* algorithm.
*/
- PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
+ PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections, BeamSqlEnv sqlEnv)
throws Exception;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
index 44e4338..f9cbf4f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
@@ -22,13 +22,13 @@ import java.io.Serializable;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSetOperatorsTransforms;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.join.CoGbkResult;
import org.apache.beam.sdk.transforms.join.CoGroupByKey;
import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
@@ -62,11 +62,11 @@ public class BeamSetOperatorRelBase {
this.all = all;
}
- public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
- PCollection<BeamSqlRow> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
+ PCollection<BeamRecord> leftRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(0))
.buildBeamPipeline(inputPCollections, sqlEnv);
- PCollection<BeamSqlRow> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
+ PCollection<BeamRecord> rightRows = BeamSqlRelUtils.getBeamRelInput(inputs.get(1))
.buildBeamPipeline(inputPCollections, sqlEnv);
WindowFn leftWindow = leftRows.getWindowingStrategy().getWindowFn();
@@ -77,20 +77,20 @@ public class BeamSetOperatorRelBase {
+ leftWindow + " VS " + rightWindow);
}
- final TupleTag<BeamSqlRow> leftTag = new TupleTag<>();
- final TupleTag<BeamSqlRow> rightTag = new TupleTag<>();
+ final TupleTag<BeamRecord> leftTag = new TupleTag<>();
+ final TupleTag<BeamRecord> rightTag = new TupleTag<>();
// co-group
String stageName = BeamSqlRelUtils.getStageName(beamRelNode);
- PCollection<KV<BeamSqlRow, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
+ PCollection<KV<BeamRecord, CoGbkResult>> coGbkResultCollection = KeyedPCollectionTuple
.of(leftTag, leftRows.apply(
stageName + "_CreateLeftIndex", MapElements.via(
new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
.and(rightTag, rightRows.apply(
stageName + "_CreateRightIndex", MapElements.via(
new BeamSetOperatorsTransforms.BeamSqlRow2KvFn())))
- .apply(CoGroupByKey.<BeamSqlRow>create());
- PCollection<BeamSqlRow> ret = coGbkResultCollection
+ .apply(CoGroupByKey.<BeamRecord>create());
+ PCollection<BeamRecord> ret = coGbkResultCollection
.apply(ParDo.of(new BeamSetOperatorsTransforms.SetOperatorFilteringDoFn(leftTag, rightTag,
opType, all)));
return ret;
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
index 4ea12ca..0cbea5c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
@@ -27,13 +27,13 @@ import java.util.List;
import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Top;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -120,10 +120,10 @@ public class BeamSortRel extends Sort implements BeamRelNode {
}
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
RelNode input = getInput();
- PCollection<BeamSqlRow> upstream = BeamSqlRelUtils.getBeamRelInput(input)
+ PCollection<BeamRecord> upstream = BeamSqlRelUtils.getBeamRelInput(input)
.buildBeamPipeline(inputPCollections, sqlEnv);
Type windowType = upstream.getWindowingStrategy().getWindowFn()
.getWindowTypeDescriptor().getType();
@@ -135,21 +135,21 @@ public class BeamSortRel extends Sort implements BeamRelNode {
BeamSqlRowComparator comparator = new BeamSqlRowComparator(fieldIndices, orientation,
nullsFirst);
// first find the top (offset + count)
- PCollection<List<BeamSqlRow>> rawStream =
+ PCollection<List<BeamRecord>> rawStream =
upstream.apply("extractTopOffsetAndFetch",
Top.of(startIndex + count, comparator).withoutDefaults())
- .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+ .setCoder(ListCoder.<BeamRecord>of(upstream.getCoder()));
// strip the `leading offset`
if (startIndex > 0) {
rawStream = rawStream.apply("stripLeadingOffset", ParDo.of(
- new SubListFn<BeamSqlRow>(startIndex, startIndex + count)))
- .setCoder(ListCoder.<BeamSqlRow>of(upstream.getCoder()));
+ new SubListFn<BeamRecord>(startIndex, startIndex + count)))
+ .setCoder(ListCoder.<BeamRecord>of(upstream.getCoder()));
}
- PCollection<BeamSqlRow> orderedStream = rawStream.apply(
- "flatten", Flatten.<BeamSqlRow>iterables());
- orderedStream.setCoder(new BeamSqlRowCoder(CalciteUtils.toBeamRowType(getRowType())));
+ PCollection<BeamRecord> orderedStream = rawStream.apply(
+ "flatten", Flatten.<BeamRecord>iterables());
+ orderedStream.setCoder(CalciteUtils.toBeamRowType(getRowType()).getRecordCoder());
return orderedStream;
}
@@ -174,7 +174,7 @@ public class BeamSortRel extends Sort implements BeamRelNode {
return new BeamSortRel(getCluster(), traitSet, newInput, newCollation, offset, fetch);
}
- private static class BeamSqlRowComparator implements Comparator<BeamSqlRow>, Serializable {
+ private static class BeamSqlRowComparator implements Comparator<BeamRecord>, Serializable {
private List<Integer> fieldsIndices;
private List<Boolean> orientation;
private List<Boolean> nullsFirst;
@@ -187,11 +187,12 @@ public class BeamSortRel extends Sort implements BeamRelNode {
this.nullsFirst = nullsFirst;
}
- @Override public int compare(BeamSqlRow row1, BeamSqlRow row2) {
+ @Override public int compare(BeamRecord row1, BeamRecord row2) {
for (int i = 0; i < fieldsIndices.size(); i++) {
int fieldIndex = fieldsIndices.get(i);
int fieldRet = 0;
- SqlTypeName fieldType = CalciteUtils.getFieldType(row1.getDataType(), fieldIndex);
+ SqlTypeName fieldType = CalciteUtils.getFieldType(
+ BeamSqlRecordHelper.getSqlRecordType(row1), fieldIndex);
// whether NULL should be ordered first or last(compared to non-null values) depends on
// what user specified in SQL(NULLS FIRST/NULLS LAST)
if (row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
@@ -203,28 +204,16 @@ public class BeamSortRel extends Sort implements BeamRelNode {
} else {
switch (fieldType) {
case TINYINT:
- fieldRet = numberCompare(row1.getByte(fieldIndex), row2.getByte(fieldIndex));
- break;
case SMALLINT:
- fieldRet = numberCompare(row1.getShort(fieldIndex), row2.getShort(fieldIndex));
- break;
case INTEGER:
- fieldRet = numberCompare(row1.getInteger(fieldIndex), row2.getInteger(fieldIndex));
- break;
case BIGINT:
- fieldRet = numberCompare(row1.getLong(fieldIndex), row2.getLong(fieldIndex));
- break;
case FLOAT:
- fieldRet = numberCompare(row1.getFloat(fieldIndex), row2.getFloat(fieldIndex));
- break;
case DOUBLE:
- fieldRet = numberCompare(row1.getDouble(fieldIndex), row2.getDouble(fieldIndex));
- break;
case VARCHAR:
- fieldRet = row1.getString(fieldIndex).compareTo(row2.getString(fieldIndex));
- break;
case DATE:
- fieldRet = row1.getDate(fieldIndex).compareTo(row2.getDate(fieldIndex));
+ Comparable v1 = (Comparable) row1.getFieldValue(fieldIndex);
+ Comparable v2 = (Comparable) row2.getFieldValue(fieldIndex);
+ fieldRet = v1.compareTo(v2);
break;
default:
throw new UnsupportedOperationException(
@@ -241,7 +230,7 @@ public class BeamSortRel extends Sort implements BeamRelNode {
}
}
- public static <T extends Number & Comparable> int numberCompare(T a, T b) {
+ public static <T extends Comparable> int compare(T a, T b) {
return a.compareTo(b);
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
index d35fa67..63ebdf3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
@@ -20,8 +20,8 @@ package org.apache.beam.sdk.extensions.sql.impl.rel;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -81,7 +81,7 @@ public class BeamUnionRel extends Union implements BeamRelNode {
return new BeamUnionRel(getCluster(), traitSet, inputs, all);
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
return delegate.buildBeamPipeline(inputPCollections, sqlEnv);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
index f12cbbc..8ad6e8d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
@@ -23,11 +23,10 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.BeamSqlEnv;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.calcite.plan.RelOptCluster;
@@ -56,17 +55,17 @@ public class BeamValuesRel extends Values implements BeamRelNode {
}
- @Override public PCollection<BeamSqlRow> buildBeamPipeline(PCollectionTuple inputPCollections
+ @Override public PCollection<BeamRecord> buildBeamPipeline(PCollectionTuple inputPCollections
, BeamSqlEnv sqlEnv) throws Exception {
- List<BeamSqlRow> rows = new ArrayList<>(tuples.size());
+ List<BeamRecord> rows = new ArrayList<>(tuples.size());
String stageName = BeamSqlRelUtils.getStageName(this);
if (tuples.isEmpty()) {
throw new IllegalStateException("Values with empty tuples!");
}
- BeamSqlRowType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
+ BeamSqlRecordType beamSQLRowType = CalciteUtils.toBeamRowType(this.getRowType());
for (ImmutableList<RexLiteral> tuple : tuples) {
- BeamSqlRow row = new BeamSqlRow(beamSQLRowType);
+ BeamRecord row = new BeamRecord(beamSQLRowType);
for (int i = 0; i < tuple.size(); i++) {
BeamTableUtils.addFieldWithAutoTypeCasting(row, i, tuple.get(i).getValue());
}
@@ -74,6 +73,6 @@ public class BeamValuesRel extends Values implements BeamRelNode {
}
return inputPCollections.getPipeline().apply(stageName, Create.of(rows))
- .setCoder(new BeamSqlRowCoder(beamSQLRowType));
+ .setCoder(beamSQLRowType.getRecordCoder());
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
index 095875f..dab79a2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java
@@ -35,13 +35,14 @@ import org.apache.beam.sdk.coders.VarIntCoder;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlInputRefExpression;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.schema.impl.AggregateFunctionImpl;
@@ -56,12 +57,12 @@ public class BeamAggregationTransforms implements Serializable{
/**
* Merge KV to single record.
*/
- public static class MergeAggregationRecord extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
- private BeamSqlRowType outRowType;
+ public static class MergeAggregationRecord extends DoFn<KV<BeamRecord, BeamRecord>, BeamRecord> {
+ private BeamSqlRecordType outRowType;
private List<String> aggFieldNames;
private int windowStartFieldIdx;
- public MergeAggregationRecord(BeamSqlRowType outRowType, List<AggregateCall> aggList
+ public MergeAggregationRecord(BeamSqlRecordType outRowType, List<AggregateCall> aggList
, int windowStartFieldIdx) {
this.outRowType = outRowType;
this.aggFieldNames = new ArrayList<>();
@@ -73,10 +74,10 @@ public class BeamAggregationTransforms implements Serializable{
@ProcessElement
public void processElement(ProcessContext c, BoundedWindow window) {
- BeamSqlRow outRecord = new BeamSqlRow(outRowType);
+ BeamRecord outRecord = new BeamRecord(outRowType);
outRecord.updateWindowRange(c.element().getKey(), window);
- KV<BeamSqlRow, BeamSqlRow> kvRecord = c.element();
+ KV<BeamRecord, BeamRecord> kvRecord = c.element();
for (String f : kvRecord.getKey().getDataType().getFieldsName()) {
outRecord.addField(f, kvRecord.getKey().getFieldValue(f));
}
@@ -95,7 +96,7 @@ public class BeamAggregationTransforms implements Serializable{
* extract group-by fields.
*/
public static class AggregationGroupByKeyFn
- implements SerializableFunction<BeamSqlRow, BeamSqlRow> {
+ implements SerializableFunction<BeamRecord, BeamRecord> {
private List<Integer> groupByKeys;
public AggregationGroupByKeyFn(int windowFieldIdx, ImmutableBitSet groupSet) {
@@ -108,9 +109,9 @@ public class BeamAggregationTransforms implements Serializable{
}
@Override
- public BeamSqlRow apply(BeamSqlRow input) {
- BeamSqlRowType typeOfKey = exTypeOfKeyRecord(input.getDataType());
- BeamSqlRow keyOfRecord = new BeamSqlRow(typeOfKey);
+ public BeamRecord apply(BeamRecord input) {
+ BeamSqlRecordType typeOfKey = exTypeOfKeyRecord(BeamSqlRecordHelper.getSqlRecordType(input));
+ BeamRecord keyOfRecord = new BeamRecord(typeOfKey);
keyOfRecord.updateWindowRange(input, null);
for (int idx = 0; idx < groupByKeys.size(); ++idx) {
@@ -119,21 +120,21 @@ public class BeamAggregationTransforms implements Serializable{
return keyOfRecord;
}
- private BeamSqlRowType exTypeOfKeyRecord(BeamSqlRowType dataType) {
+ private BeamSqlRecordType exTypeOfKeyRecord(BeamSqlRecordType dataType) {
List<String> fieldNames = new ArrayList<>();
List<Integer> fieldTypes = new ArrayList<>();
for (int idx : groupByKeys) {
fieldNames.add(dataType.getFieldsName().get(idx));
fieldTypes.add(dataType.getFieldsType().get(idx));
}
- return BeamSqlRowType.create(fieldNames, fieldTypes);
+ return BeamSqlRecordType.create(fieldNames, fieldTypes);
}
}
/**
* Assign event timestamp.
*/
- public static class WindowTimestampFn implements SerializableFunction<BeamSqlRow, Instant> {
+ public static class WindowTimestampFn implements SerializableFunction<BeamRecord, Instant> {
private int windowFieldIdx = -1;
public WindowTimestampFn(int windowFieldIdx) {
@@ -142,7 +143,7 @@ public class BeamAggregationTransforms implements Serializable{
}
@Override
- public Instant apply(BeamSqlRow input) {
+ public Instant apply(BeamRecord input) {
return new Instant(input.getDate(windowFieldIdx).getTime());
}
}
@@ -151,13 +152,13 @@ public class BeamAggregationTransforms implements Serializable{
* An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}.
*/
public static class AggregationAdaptor
- extends CombineFn<BeamSqlRow, AggregationAccumulator, BeamSqlRow> {
+ extends CombineFn<BeamRecord, AggregationAccumulator, BeamRecord> {
private List<BeamSqlUdaf> aggregators;
private List<BeamSqlExpression> sourceFieldExps;
- private BeamSqlRowType finalRowType;
+ private BeamSqlRecordType finalRowType;
public AggregationAdaptor(List<AggregateCall> aggregationCalls,
- BeamSqlRowType sourceRowType) {
+ BeamSqlRecordType sourceRowType) {
aggregators = new ArrayList<>();
sourceFieldExps = new ArrayList<>();
List<String> outFieldsName = new ArrayList<>();
@@ -206,7 +207,7 @@ public class BeamAggregationTransforms implements Serializable{
break;
}
}
- finalRowType = BeamSqlRowType.create(outFieldsName, outFieldsType);
+ finalRowType = BeamSqlRecordType.create(outFieldsName, outFieldsType);
}
@Override
public AggregationAccumulator createAccumulator() {
@@ -217,7 +218,7 @@ public class BeamAggregationTransforms implements Serializable{
return initialAccu;
}
@Override
- public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamSqlRow input) {
+ public AggregationAccumulator addInput(AggregationAccumulator accumulator, BeamRecord input) {
AggregationAccumulator deltaAcc = new AggregationAccumulator();
for (int idx = 0; idx < aggregators.size(); ++idx) {
deltaAcc.accumulatorElements.add(
@@ -240,8 +241,8 @@ public class BeamAggregationTransforms implements Serializable{
return deltaAcc;
}
@Override
- public BeamSqlRow extractOutput(AggregationAccumulator accumulator) {
- BeamSqlRow result = new BeamSqlRow(finalRowType);
+ public BeamRecord extractOutput(AggregationAccumulator accumulator) {
+ BeamRecord result = new BeamRecord(finalRowType);
for (int idx = 0; idx < aggregators.size(); ++idx) {
result.addField(idx, aggregators.get(idx).result(accumulator.accumulatorElements.get(idx)));
}
@@ -249,7 +250,7 @@ public class BeamAggregationTransforms implements Serializable{
}
@Override
public Coder<AggregationAccumulator> getAccumulatorCoder(
- CoderRegistry registry, Coder<BeamSqlRow> inputCoder)
+ CoderRegistry registry, Coder<BeamRecord> inputCoder)
throws CannotProvideCoderException {
registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of());
List<Coder> aggAccuCoderList = new ArrayList<>();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
index e0898d1..105bbf3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
@@ -22,10 +22,11 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordHelper;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.calcite.rel.core.JoinRelType;
@@ -40,7 +41,7 @@ public class BeamJoinTransforms {
* A {@code SimpleFunction} to extract join fields from the specified row.
*/
public static class ExtractJoinFields
- extends SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
+ extends SimpleFunction<BeamRecord, KV<BeamRecord, BeamRecord>> {
private final boolean isLeft;
private final List<Pair<Integer, Integer>> joinColumns;
@@ -49,7 +50,7 @@ public class BeamJoinTransforms {
this.joinColumns = joinColumns;
}
- @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+ @Override public KV<BeamRecord, BeamRecord> apply(BeamRecord input) {
// build the type
// the name of the join field is not important
List<String> names = new ArrayList<>(joinColumns.size());
@@ -57,13 +58,15 @@ public class BeamJoinTransforms {
for (int i = 0; i < joinColumns.size(); i++) {
names.add("c" + i);
types.add(isLeft
- ? input.getDataType().getFieldsType().get(joinColumns.get(i).getKey()) :
- input.getDataType().getFieldsType().get(joinColumns.get(i).getValue()));
+ ? BeamSqlRecordHelper.getSqlRecordType(input).getFieldsType()
+ .get(joinColumns.get(i).getKey())
+ : BeamSqlRecordHelper.getSqlRecordType(input).getFieldsType()
+ .get(joinColumns.get(i).getValue()));
}
- BeamSqlRowType type = BeamSqlRowType.create(names, types);
+ BeamSqlRecordType type = BeamSqlRecordType.create(names, types);
// build the row
- BeamSqlRow row = new BeamSqlRow(type);
+ BeamRecord row = new BeamRecord(type);
for (int i = 0; i < joinColumns.size(); i++) {
row.addField(i, input
.getFieldValue(isLeft ? joinColumns.get(i).getKey() : joinColumns.get(i).getValue()));
@@ -76,14 +79,14 @@ public class BeamJoinTransforms {
/**
* A {@code DoFn} which implement the sideInput-JOIN.
*/
- public static class SideInputJoinDoFn extends DoFn<KV<BeamSqlRow, BeamSqlRow>, BeamSqlRow> {
- private final PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView;
+ public static class SideInputJoinDoFn extends DoFn<KV<BeamRecord, BeamRecord>, BeamRecord> {
+ private final PCollectionView<Map<BeamRecord, Iterable<BeamRecord>>> sideInputView;
private final JoinRelType joinType;
- private final BeamSqlRow rightNullRow;
+ private final BeamRecord rightNullRow;
private final boolean swap;
- public SideInputJoinDoFn(JoinRelType joinType, BeamSqlRow rightNullRow,
- PCollectionView<Map<BeamSqlRow, Iterable<BeamSqlRow>>> sideInputView,
+ public SideInputJoinDoFn(JoinRelType joinType, BeamRecord rightNullRow,
+ PCollectionView<Map<BeamRecord, Iterable<BeamRecord>>> sideInputView,
boolean swap) {
this.joinType = joinType;
this.rightNullRow = rightNullRow;
@@ -92,13 +95,13 @@ public class BeamJoinTransforms {
}
@ProcessElement public void processElement(ProcessContext context) {
- BeamSqlRow key = context.element().getKey();
- BeamSqlRow leftRow = context.element().getValue();
- Map<BeamSqlRow, Iterable<BeamSqlRow>> key2Rows = context.sideInput(sideInputView);
- Iterable<BeamSqlRow> rightRowsIterable = key2Rows.get(key);
+ BeamRecord key = context.element().getKey();
+ BeamRecord leftRow = context.element().getValue();
+ Map<BeamRecord, Iterable<BeamRecord>> key2Rows = context.sideInput(sideInputView);
+ Iterable<BeamRecord> rightRowsIterable = key2Rows.get(key);
if (rightRowsIterable != null && rightRowsIterable.iterator().hasNext()) {
- Iterator<BeamSqlRow> it = rightRowsIterable.iterator();
+ Iterator<BeamRecord> it = rightRowsIterable.iterator();
while (it.hasNext()) {
context.output(combineTwoRowsIntoOne(leftRow, it.next(), swap));
}
@@ -115,11 +118,11 @@ public class BeamJoinTransforms {
* A {@code SimpleFunction} to combine two rows into one.
*/
public static class JoinParts2WholeRow
- extends SimpleFunction<KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>>, BeamSqlRow> {
- @Override public BeamSqlRow apply(KV<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> input) {
- KV<BeamSqlRow, BeamSqlRow> parts = input.getValue();
- BeamSqlRow leftRow = parts.getKey();
- BeamSqlRow rightRow = parts.getValue();
+ extends SimpleFunction<KV<BeamRecord, KV<BeamRecord, BeamRecord>>, BeamRecord> {
+ @Override public BeamRecord apply(KV<BeamRecord, KV<BeamRecord, BeamRecord>> input) {
+ KV<BeamRecord, BeamRecord> parts = input.getValue();
+ BeamRecord leftRow = parts.getKey();
+ BeamRecord rightRow = parts.getValue();
return combineTwoRowsIntoOne(leftRow, rightRow, false);
}
}
@@ -127,8 +130,8 @@ public class BeamJoinTransforms {
/**
* As the method name suggests: combine two rows into one wide row.
*/
- private static BeamSqlRow combineTwoRowsIntoOne(BeamSqlRow leftRow,
- BeamSqlRow rightRow, boolean swap) {
+ private static BeamRecord combineTwoRowsIntoOne(BeamRecord leftRow,
+ BeamRecord rightRow, boolean swap) {
if (swap) {
return combineTwoRowsIntoOneHelper(rightRow, leftRow);
} else {
@@ -139,19 +142,19 @@ public class BeamJoinTransforms {
/**
* As the method name suggests: combine two rows into one wide row.
*/
- private static BeamSqlRow combineTwoRowsIntoOneHelper(BeamSqlRow leftRow,
- BeamSqlRow rightRow) {
+ private static BeamRecord combineTwoRowsIntoOneHelper(BeamRecord leftRow,
+ BeamRecord rightRow) {
// build the type
List<String> names = new ArrayList<>(leftRow.size() + rightRow.size());
names.addAll(leftRow.getDataType().getFieldsName());
names.addAll(rightRow.getDataType().getFieldsName());
List<Integer> types = new ArrayList<>(leftRow.size() + rightRow.size());
- types.addAll(leftRow.getDataType().getFieldsType());
- types.addAll(rightRow.getDataType().getFieldsType());
- BeamSqlRowType type = BeamSqlRowType.create(names, types);
+ types.addAll(BeamSqlRecordHelper.getSqlRecordType(leftRow).getFieldsType());
+ types.addAll(BeamSqlRecordHelper.getSqlRecordType(rightRow).getFieldsType());
+ BeamSqlRecordType type = BeamSqlRecordType.create(names, types);
- BeamSqlRow row = new BeamSqlRow(type);
+ BeamRecord row = new BeamRecord(type);
// build the row
for (int i = 0; i < leftRow.size(); i++) {
row.addField(i, leftRow.getFieldValue(i));
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
index 326b328..33ac807 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
@@ -20,10 +20,10 @@ package org.apache.beam.sdk.extensions.sql.impl.transform;
import java.util.Iterator;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSetOperatorRelBase;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.join.CoGbkResult;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.TupleTag;
@@ -35,8 +35,8 @@ public abstract class BeamSetOperatorsTransforms {
* Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}.
*/
public static class BeamSqlRow2KvFn extends
- SimpleFunction<BeamSqlRow, KV<BeamSqlRow, BeamSqlRow>> {
- @Override public KV<BeamSqlRow, BeamSqlRow> apply(BeamSqlRow input) {
+ SimpleFunction<BeamRecord, KV<BeamRecord, BeamRecord>> {
+ @Override public KV<BeamRecord, BeamRecord> apply(BeamRecord input) {
return KV.of(input, input);
}
}
@@ -45,14 +45,14 @@ public abstract class BeamSetOperatorsTransforms {
* Filter function used for Set operators.
*/
public static class SetOperatorFilteringDoFn extends
- DoFn<KV<BeamSqlRow, CoGbkResult>, BeamSqlRow> {
- private TupleTag<BeamSqlRow> leftTag;
- private TupleTag<BeamSqlRow> rightTag;
+ DoFn<KV<BeamRecord, CoGbkResult>, BeamRecord> {
+ private TupleTag<BeamRecord> leftTag;
+ private TupleTag<BeamRecord> rightTag;
private BeamSetOperatorRelBase.OpType opType;
// ALL?
private boolean all;
- public SetOperatorFilteringDoFn(TupleTag<BeamSqlRow> leftTag, TupleTag<BeamSqlRow> rightTag,
+ public SetOperatorFilteringDoFn(TupleTag<BeamRecord> leftTag, TupleTag<BeamRecord> rightTag,
BeamSetOperatorRelBase.OpType opType, boolean all) {
this.leftTag = leftTag;
this.rightTag = rightTag;
@@ -62,13 +62,13 @@ public abstract class BeamSetOperatorsTransforms {
@ProcessElement public void processElement(ProcessContext ctx) {
CoGbkResult coGbkResult = ctx.element().getValue();
- Iterable<BeamSqlRow> leftRows = coGbkResult.getAll(leftTag);
- Iterable<BeamSqlRow> rightRows = coGbkResult.getAll(rightTag);
+ Iterable<BeamRecord> leftRows = coGbkResult.getAll(leftTag);
+ Iterable<BeamRecord> rightRows = coGbkResult.getAll(rightTag);
switch (opType) {
case UNION:
if (all) {
// output both left & right
- Iterator<BeamSqlRow> iter = leftRows.iterator();
+ Iterator<BeamRecord> iter = leftRows.iterator();
while (iter.hasNext()) {
ctx.output(iter.next());
}
@@ -84,7 +84,7 @@ public abstract class BeamSetOperatorsTransforms {
case INTERSECT:
if (leftRows.iterator().hasNext() && rightRows.iterator().hasNext()) {
if (all) {
- for (BeamSqlRow leftRow : leftRows) {
+ for (BeamRecord leftRow : leftRows) {
ctx.output(leftRow);
}
} else {
@@ -94,7 +94,7 @@ public abstract class BeamSetOperatorsTransforms {
break;
case MINUS:
if (leftRows.iterator().hasNext() && !rightRows.iterator().hasNext()) {
- Iterator<BeamSqlRow> iter = leftRows.iterator();
+ Iterator<BeamRecord> iter = leftRows.iterator();
if (all) {
// output all
while (iter.hasNext()) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
index 855de7a..31efeb7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlFilterFn.java
@@ -20,14 +20,14 @@ package org.apache.beam.sdk.extensions.sql.impl.transform;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.BeamRecord;
/**
* {@code BeamSqlFilterFn} is the executor for a {@link BeamFilterRel} step.
*
*/
-public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+public class BeamSqlFilterFn extends DoFn<BeamRecord, BeamRecord> {
private String stepName;
private BeamSqlExpressionExecutor executor;
@@ -45,7 +45,7 @@ public class BeamSqlFilterFn extends DoFn<BeamSqlRow, BeamSqlRow> {
@ProcessElement
public void processElement(ProcessContext c) {
- BeamSqlRow in = c.element();
+ BeamRecord in = c.element();
List<Object> result = executor.execute(in);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
index b40cfa6..f97a90a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlOutputToConsoleFn.java
@@ -17,14 +17,14 @@
*/
package org.apache.beam.sdk.extensions.sql.impl.transform;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.values.BeamRecord;
/**
* A test PTransform to display output in console.
*
*/
-public class BeamSqlOutputToConsoleFn extends DoFn<BeamSqlRow, Void> {
+public class BeamSqlOutputToConsoleFn extends DoFn<BeamRecord, Void> {
private String stepName;
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
index b3f7ce5..a95c743 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSqlProjectFn.java
@@ -20,24 +20,24 @@ package org.apache.beam.sdk.extensions.sql.impl.transform;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamTableUtils;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.BeamRecord;
/**
*
* {@code BeamSqlProjectFn} is the executor for a {@link BeamProjectRel} step.
*
*/
-public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
+public class BeamSqlProjectFn extends DoFn<BeamRecord, BeamRecord> {
private String stepName;
private BeamSqlExpressionExecutor executor;
- private BeamSqlRowType outputRowType;
+ private BeamSqlRecordType outputRowType;
public BeamSqlProjectFn(String stepName, BeamSqlExpressionExecutor executor,
- BeamSqlRowType outputRowType) {
+ BeamSqlRecordType outputRowType) {
super();
this.stepName = stepName;
this.executor = executor;
@@ -51,10 +51,10 @@ public class BeamSqlProjectFn extends DoFn<BeamSqlRow, BeamSqlRow> {
@ProcessElement
public void processElement(ProcessContext c, BoundedWindow window) {
- BeamSqlRow inputRow = c.element();
+ BeamRecord inputRow = c.element();
List<Object> results = executor.execute(inputRow);
- BeamSqlRow outRow = new BeamSqlRow(outputRowType);
+ BeamRecord outRow = new BeamRecord(outputRowType);
outRow.updateWindowRange(inputRow, window);
for (int idx = 0; idx < results.size(); ++idx) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
index b80e045..bf96e85 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
@@ -78,27 +78,27 @@ public class CalciteUtils {
/**
* Get the {@code SqlTypeName} for the specified column of a table.
*/
- public static SqlTypeName getFieldType(BeamSqlRowType schema, int index) {
+ public static SqlTypeName getFieldType(BeamSqlRecordType schema, int index) {
return toCalciteType(schema.getFieldsType().get(index));
}
/**
* Generate {@code BeamSqlRowType} from {@code RelDataType} which is used to create table.
*/
- public static BeamSqlRowType toBeamRowType(RelDataType tableInfo) {
+ public static BeamSqlRecordType toBeamRowType(RelDataType tableInfo) {
List<String> fieldNames = new ArrayList<>();
List<Integer> fieldTypes = new ArrayList<>();
for (RelDataTypeField f : tableInfo.getFieldList()) {
fieldNames.add(f.getName());
fieldTypes.add(toJavaType(f.getType().getSqlTypeName()));
}
- return BeamSqlRowType.create(fieldNames, fieldTypes);
+ return BeamSqlRecordType.create(fieldNames, fieldTypes);
}
/**
* Create an instance of {@code RelDataType} so it can be used to create a table.
*/
- public static RelProtoDataType toCalciteRowType(final BeamSqlRowType that) {
+ public static RelProtoDataType toCalciteRowType(final BeamSqlRecordType that) {
return new RelProtoDataType() {
@Override
public RelDataType apply(RelDataTypeFactory a) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
index bf41c95..68b120e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BaseBeamTable.java
@@ -23,12 +23,12 @@ import java.io.Serializable;
* Each IO in Beam has one table schema, by extending {@link BaseBeamTable}.
*/
public abstract class BaseBeamTable implements BeamSqlTable, Serializable {
- protected BeamSqlRowType beamSqlRowType;
- public BaseBeamTable(BeamSqlRowType beamSqlRowType) {
+ protected BeamSqlRecordType beamSqlRowType;
+ public BaseBeamTable(BeamSqlRecordType beamSqlRowType) {
this.beamSqlRowType = beamSqlRowType;
}
- @Override public BeamSqlRowType getRowType() {
+ @Override public BeamSqlRecordType getRowType() {
return beamSqlRowType;
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
index 5bbb8fd..68905b5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamPCollectionTable.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.extensions.sql.schema;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.PDone;
@@ -29,14 +30,14 @@ import org.apache.beam.sdk.values.PDone;
*/
public class BeamPCollectionTable extends BaseBeamTable {
private BeamIOType ioType;
- private transient PCollection<BeamSqlRow> upstream;
+ private transient PCollection<BeamRecord> upstream;
- protected BeamPCollectionTable(BeamSqlRowType beamSqlRowType) {
+ protected BeamPCollectionTable(BeamSqlRecordType beamSqlRowType) {
super(beamSqlRowType);
}
- public BeamPCollectionTable(PCollection<BeamSqlRow> upstream,
- BeamSqlRowType beamSqlRowType){
+ public BeamPCollectionTable(PCollection<BeamRecord> upstream,
+ BeamSqlRecordType beamSqlRowType){
this(beamSqlRowType);
ioType = upstream.isBounded().equals(IsBounded.BOUNDED)
? BeamIOType.BOUNDED : BeamIOType.UNBOUNDED;
@@ -49,12 +50,12 @@ public class BeamPCollectionTable extends BaseBeamTable {
}
@Override
- public PCollection<BeamSqlRow> buildIOReader(Pipeline pipeline) {
+ public PCollection<BeamRecord> buildIOReader(Pipeline pipeline) {
return upstream;
}
@Override
- public PTransform<? super PCollection<BeamSqlRow>, PDone> buildIOWriter() {
+ public PTransform<? super PCollection<BeamRecord>, PDone> buildIOWriter() {
throw new IllegalArgumentException("cannot use [BeamPCollectionTable] as target");
}
[2/7] beam git commit: use BitSet for nullFields
Posted by ro...@apache.org.
use BitSet for nullFields
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/129ae969
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/129ae969
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/129ae969
Branch: refs/heads/DSL_SQL
Commit: 129ae9696af6a2f8d83ee962ca2ba8a7d6e3fd40
Parents: 52933a6
Author: mingmxu <mi...@ebay.com>
Authored: Thu Aug 3 00:43:46 2017 -0700
Committer: mingmxu <mi...@ebay.com>
Committed: Thu Aug 3 00:43:46 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 18 +++++++++++----
.../org/apache/beam/sdk/values/BeamRecord.java | 24 ++++++++++----------
.../extensions/sql/schema/BeamSqlRowCoder.java | 14 ++++++------
3 files changed, 32 insertions(+), 24 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/beam/blob/129ae969/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
index ad27f4e..27f92ce 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.coders;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
+import java.util.BitSet;
import java.util.List;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.values.BeamRecord;
@@ -30,23 +31,30 @@ import org.apache.beam.sdk.values.BeamRecordTypeProvider;
*/
@Experimental
public class BeamRecordCoder extends CustomCoder<BeamRecord> {
- private static final ListCoder<Integer> nullListCoder = ListCoder.of(BigEndianIntegerCoder.of());
+ private static final BitSetCoder nullListCoder = BitSetCoder.of();
private static final InstantCoder instantCoder = InstantCoder.of();
private BeamRecordTypeProvider recordType;
private List<Coder> coderArray;
- public BeamRecordCoder(BeamRecordTypeProvider recordType, List<Coder> coderArray) {
+ private BeamRecordCoder(BeamRecordTypeProvider recordType, List<Coder> coderArray) {
this.recordType = recordType;
this.coderArray = coderArray;
}
+ public static BeamRecordCoder of(BeamRecordTypeProvider recordType, List<Coder> coderArray){
+ if (recordType.size() != coderArray.size()) {
+ throw new IllegalArgumentException("Coder size doesn't match with field size");
+ }
+ return new BeamRecordCoder(recordType, coderArray);
+ }
+
@Override
public void encode(BeamRecord value, OutputStream outStream)
throws CoderException, IOException {
nullListCoder.encode(value.getNullFields(), outStream);
for (int idx = 0; idx < value.size(); ++idx) {
- if (value.getNullFields().contains(idx)) {
+ if (value.getNullFields().get(idx)) {
continue;
}
@@ -59,12 +67,12 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
@Override
public BeamRecord decode(InputStream inStream) throws CoderException, IOException {
- List<Integer> nullFields = nullListCoder.decode(inStream);
+ BitSet nullFields = nullListCoder.decode(inStream);
BeamRecord record = new BeamRecord(recordType);
record.setNullFields(nullFields);
for (int idx = 0; idx < recordType.size(); ++idx) {
- if (nullFields.contains(idx)) {
+ if (nullFields.get(idx)) {
continue;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/129ae969/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
index d1c1c17..476233e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.values;
import java.io.Serializable;
import java.math.BigDecimal;
import java.util.ArrayList;
+import java.util.BitSet;
import java.util.Date;
import java.util.GregorianCalendar;
import java.util.List;
@@ -36,9 +37,9 @@ import org.joda.time.Instant;
*/
@Experimental
public class BeamRecord implements Serializable {
- //null values are indexed here, to handle properly in Coder.
- private List<Integer> nullFields = new ArrayList<>();
private List<Object> dataValues;
+ //null values are indexed here, to handle properly in Coder.
+ private BitSet nullFields;
private BeamRecordTypeProvider dataType;
private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
@@ -46,10 +47,11 @@ public class BeamRecord implements Serializable {
public BeamRecord(BeamRecordTypeProvider dataType) {
this.dataType = dataType;
+ this.nullFields = new BitSet(dataType.size());
this.dataValues = new ArrayList<>();
for (int idx = 0; idx < dataType.size(); ++idx) {
dataValues.add(null);
- nullFields.add(idx);
+ nullFields.set(idx);
}
}
@@ -79,9 +81,7 @@ public class BeamRecord implements Serializable {
if (fieldValue == null) {
return;
} else {
- if (nullFields.contains(index)) {
- nullFields.remove(nullFields.indexOf(index));
- }
+ nullFields.clear(index);
}
dataType.validateValueType(index, fieldValue);
@@ -137,7 +137,7 @@ public class BeamRecord implements Serializable {
}
public Object getFieldValue(int fieldIdx) {
- if (nullFields.contains(fieldIdx)) {
+ if (nullFields.get(fieldIdx)) {
return null;
}
@@ -208,19 +208,19 @@ public class BeamRecord implements Serializable {
this.dataType = dataType;
}
- public void setNullFields(List<Integer> nullFields) {
- this.nullFields = nullFields;
+ public BitSet getNullFields() {
+ return nullFields;
}
- public List<Integer> getNullFields() {
- return nullFields;
+ public void setNullFields(BitSet nullFields) {
+ this.nullFields = nullFields;
}
/**
* is the specified field NULL?
*/
public boolean isNull(int idx) {
- return nullFields.contains(idx);
+ return nullFields.get(idx);
}
public Instant getWindowStart() {
http://git-wip-us.apache.org/repos/asf/beam/blob/129ae969/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
index 3d760c4..c7656af 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/schema/BeamSqlRowCoder.java
@@ -21,19 +21,19 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.sql.Types;
+import java.util.BitSet;
import java.util.Date;
import java.util.GregorianCalendar;
-import java.util.List;
import org.apache.beam.sdk.coders.BigDecimalCoder;
import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
import org.apache.beam.sdk.coders.BigEndianLongCoder;
+import org.apache.beam.sdk.coders.BitSetCoder;
import org.apache.beam.sdk.coders.ByteCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.DoubleCoder;
import org.apache.beam.sdk.coders.InstantCoder;
-import org.apache.beam.sdk.coders.ListCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
/**
@@ -42,7 +42,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder;
public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
private BeamSqlRowType sqlRecordType;
- private static final ListCoder<Integer> listCoder = ListCoder.of(BigEndianIntegerCoder.of());
+ private static final BitSetCoder nullListCoder = BitSetCoder.of();
private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of();
@@ -59,9 +59,9 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
@Override
public void encode(BeamSqlRow value, OutputStream outStream)
throws CoderException, IOException {
- listCoder.encode(value.getNullFields(), outStream);
+ nullListCoder.encode(value.getNullFields(), outStream);
for (int idx = 0; idx < value.size(); ++idx) {
- if (value.getNullFields().contains(idx)) {
+ if (value.getNullFields().get(idx)) {
continue;
}
@@ -114,12 +114,12 @@ public class BeamSqlRowCoder extends CustomCoder<BeamSqlRow> {
@Override
public BeamSqlRow decode(InputStream inStream) throws CoderException, IOException {
- List<Integer> nullFields = listCoder.decode(inStream);
+ BitSet nullFields = nullListCoder.decode(inStream);
BeamSqlRow record = new BeamSqlRow(sqlRecordType);
record.setNullFields(nullFields);
for (int idx = 0; idx < sqlRecordType.size(); ++idx) {
- if (nullFields.contains(idx)) {
+ if (nullFields.get(idx)) {
continue;
}
[6/7] beam git commit: refactor BeamRecord, BeamRecordType,
BeamSqlRecordType, BeamRecordCoder
Posted by ro...@apache.org.
refactor BeamRecord, BeamRecordType, BeamSqlRecordType, BeamRecordCoder
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/89109b8c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/89109b8c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/89109b8c
Branch: refs/heads/DSL_SQL
Commit: 89109b8cdc667c4e07529e9748ed4290e88b9282
Parents: 129ae96
Author: mingmxu <mi...@ebay.com>
Authored: Thu Aug 3 12:11:06 2017 -0700
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Aug 4 10:08:37 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 20 +-
.../org/apache/beam/sdk/values/BeamRecord.java | 22 +-
.../apache/beam/sdk/values/BeamRecordType.java | 70 ++++++
.../beam/sdk/values/BeamRecordTypeProvider.java | 59 -----
.../apache/beam/sdk/extensions/sql/BeamSql.java | 22 +-
.../beam/sdk/extensions/sql/BeamSqlCli.java | 8 +-
.../beam/sdk/extensions/sql/BeamSqlEnv.java | 6 +-
.../extensions/sql/example/BeamSqlExample.java | 27 ++-
.../interpreter/BeamSqlExpressionExecutor.java | 6 +-
.../sql/impl/interpreter/BeamSqlFnExecutor.java | 6 +-
.../operator/BeamSqlCaseExpression.java | 4 +-
.../operator/BeamSqlCastExpression.java | 4 +-
.../interpreter/operator/BeamSqlExpression.java | 8 +-
.../operator/BeamSqlInputRefExpression.java | 4 +-
.../interpreter/operator/BeamSqlPrimitive.java | 6 +-
.../operator/BeamSqlReinterpretExpression.java | 4 +-
.../operator/BeamSqlUdfExpression.java | 4 +-
.../operator/BeamSqlWindowEndExpression.java | 4 +-
.../operator/BeamSqlWindowExpression.java | 4 +-
.../operator/BeamSqlWindowStartExpression.java | 4 +-
.../arithmetic/BeamSqlArithmeticExpression.java | 4 +-
.../comparison/BeamSqlCompareExpression.java | 4 +-
.../comparison/BeamSqlIsNotNullExpression.java | 4 +-
.../comparison/BeamSqlIsNullExpression.java | 4 +-
.../date/BeamSqlCurrentDateExpression.java | 4 +-
.../date/BeamSqlCurrentTimeExpression.java | 4 +-
.../date/BeamSqlCurrentTimestampExpression.java | 4 +-
.../date/BeamSqlDateCeilExpression.java | 4 +-
.../date/BeamSqlDateFloorExpression.java | 4 +-
.../operator/date/BeamSqlExtractExpression.java | 4 +-
.../operator/logical/BeamSqlAndExpression.java | 4 +-
.../operator/logical/BeamSqlNotExpression.java | 4 +-
.../operator/logical/BeamSqlOrExpression.java | 4 +-
.../math/BeamSqlMathBinaryExpression.java | 4 +-
.../math/BeamSqlMathUnaryExpression.java | 4 +-
.../operator/math/BeamSqlPiExpression.java | 4 +-
.../operator/math/BeamSqlRandExpression.java | 4 +-
.../math/BeamSqlRandIntegerExpression.java | 4 +-
.../string/BeamSqlCharLengthExpression.java | 4 +-
.../string/BeamSqlConcatExpression.java | 4 +-
.../string/BeamSqlInitCapExpression.java | 4 +-
.../operator/string/BeamSqlLowerExpression.java | 4 +-
.../string/BeamSqlOverlayExpression.java | 4 +-
.../string/BeamSqlPositionExpression.java | 4 +-
.../string/BeamSqlSubstringExpression.java | 4 +-
.../operator/string/BeamSqlTrimExpression.java | 4 +-
.../operator/string/BeamSqlUpperExpression.java | 4 +-
.../sql/impl/planner/BeamQueryPlanner.java | 4 +-
.../sql/impl/rel/BeamAggregationRel.java | 38 ++--
.../extensions/sql/impl/rel/BeamFilterRel.java | 11 +-
.../extensions/sql/impl/rel/BeamIOSinkRel.java | 6 +-
.../sql/impl/rel/BeamIOSourceRel.java | 13 +-
.../sql/impl/rel/BeamIntersectRel.java | 4 +-
.../extensions/sql/impl/rel/BeamJoinRel.java | 75 ++++---
.../extensions/sql/impl/rel/BeamMinusRel.java | 4 +-
.../extensions/sql/impl/rel/BeamProjectRel.java | 11 +-
.../extensions/sql/impl/rel/BeamRelNode.java | 4 +-
.../sql/impl/rel/BeamSetOperatorRelBase.java | 18 +-
.../extensions/sql/impl/rel/BeamSortRel.java | 49 ++---
.../extensions/sql/impl/rel/BeamUnionRel.java | 4 +-
.../extensions/sql/impl/rel/BeamValuesRel.java | 15 +-
.../transform/BeamAggregationTransforms.java | 47 ++--
.../sql/impl/transform/BeamJoinTransforms.java | 65 +++---
.../transform/BeamSetOperatorsTransforms.java | 24 +-
.../sql/impl/transform/BeamSqlFilterFn.java | 6 +-
.../transform/BeamSqlOutputToConsoleFn.java | 4 +-
.../sql/impl/transform/BeamSqlProjectFn.java | 14 +-
.../extensions/sql/impl/utils/CalciteUtils.java | 10 +-
.../extensions/sql/schema/BaseBeamTable.java | 6 +-
.../sql/schema/BeamPCollectionTable.java | 13 +-
.../sql/schema/BeamSqlRecordHelper.java | 217 +++++++++++++++++++
.../sql/schema/BeamSqlRecordType.java | 168 ++++++++++++++
.../sdk/extensions/sql/schema/BeamSqlRow.java | 41 ----
.../extensions/sql/schema/BeamSqlRowCoder.java | 186 ----------------
.../extensions/sql/schema/BeamSqlRowType.java | 109 ----------
.../sdk/extensions/sql/schema/BeamSqlTable.java | 7 +-
.../extensions/sql/schema/BeamTableUtils.java | 14 +-
.../sql/schema/kafka/BeamKafkaCSVTable.java | 38 ++--
.../sql/schema/kafka/BeamKafkaTable.java | 20 +-
.../sql/schema/text/BeamTextCSVTable.java | 12 +-
.../schema/text/BeamTextCSVTableIOReader.java | 14 +-
.../schema/text/BeamTextCSVTableIOWriter.java | 16 +-
.../sql/schema/text/BeamTextTable.java | 4 +-
.../sql/BeamSqlDslAggregationTest.java | 80 +++----
.../beam/sdk/extensions/sql/BeamSqlDslBase.java | 51 +++--
.../extensions/sql/BeamSqlDslFilterTest.java | 26 +--
.../sdk/extensions/sql/BeamSqlDslJoinTest.java | 26 +--
.../extensions/sql/BeamSqlDslProjectTest.java | 64 +++---
.../extensions/sql/BeamSqlDslUdfUdafTest.java | 24 +-
.../beam/sdk/extensions/sql/TestUtils.java | 30 +--
.../interpreter/BeamSqlFnExecutorTestBase.java | 10 +-
.../sql/impl/rel/BeamIntersectRelTest.java | 6 +-
.../rel/BeamJoinRelBoundedVsBoundedTest.java | 10 +-
.../rel/BeamJoinRelUnboundedVsBoundedTest.java | 10 +-
.../BeamJoinRelUnboundedVsUnboundedTest.java | 10 +-
.../sql/impl/rel/BeamMinusRelTest.java | 6 +-
.../impl/rel/BeamSetOperatorRelBaseTest.java | 4 +-
.../sql/impl/rel/BeamSortRelTest.java | 12 +-
.../sql/impl/rel/BeamUnionRelTest.java | 6 +-
.../sql/impl/rel/BeamValuesRelTest.java | 8 +-
.../sdk/extensions/sql/impl/rel/CheckSize.java | 8 +-
...mSqlBuiltinFunctionsIntegrationTestBase.java | 17 +-
...amSqlComparisonOperatorsIntegrationTest.java | 11 +-
.../BeamSqlDateFunctionsIntegrationTest.java | 12 +-
.../extensions/sql/mock/MockedBoundedTable.java | 24 +-
.../sdk/extensions/sql/mock/MockedTable.java | 8 +-
.../sql/mock/MockedUnboundedTable.java | 18 +-
.../sql/schema/BeamSqlRowCoderTest.java | 8 +-
.../sql/schema/kafka/BeamKafkaCSVTableTest.java | 14 +-
.../sql/schema/text/BeamTextCSVTableTest.java | 16 +-
.../transform/BeamAggregationTransformTest.java | 72 +++---
.../schema/transform/BeamTransformBaseTest.java | 18 +-
112 files changed, 1171 insertions(+), 1129 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
index 27f92ce..06958a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BeamRecordCoder.java
@@ -24,7 +24,7 @@ import java.util.BitSet;
import java.util.List;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.values.BeamRecord;
-import org.apache.beam.sdk.values.BeamRecordTypeProvider;
+import org.apache.beam.sdk.values.BeamRecordType;
/**
* A {@link Coder} for {@link BeamRecord}. It wraps the {@link Coder} for each element directly.
@@ -34,31 +34,35 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
private static final BitSetCoder nullListCoder = BitSetCoder.of();
private static final InstantCoder instantCoder = InstantCoder.of();
- private BeamRecordTypeProvider recordType;
+ private BeamRecordType recordType;
private List<Coder> coderArray;
- private BeamRecordCoder(BeamRecordTypeProvider recordType, List<Coder> coderArray) {
+ private BeamRecordCoder(BeamRecordType recordType, List<Coder> coderArray) {
this.recordType = recordType;
this.coderArray = coderArray;
}
- public static BeamRecordCoder of(BeamRecordTypeProvider recordType, List<Coder> coderArray){
+ public static BeamRecordCoder of(BeamRecordType recordType, List<Coder> coderArray){
if (recordType.size() != coderArray.size()) {
throw new IllegalArgumentException("Coder size doesn't match with field size");
}
return new BeamRecordCoder(recordType, coderArray);
}
+ public BeamRecordType getRecordType() {
+ return recordType;
+ }
+
@Override
public void encode(BeamRecord value, OutputStream outStream)
throws CoderException, IOException {
nullListCoder.encode(value.getNullFields(), outStream);
for (int idx = 0; idx < value.size(); ++idx) {
- if (value.getNullFields().get(idx)) {
+ if (value.isNull(idx)) {
continue;
}
- coderArray.get(idx).encode(value.getInteger(idx), outStream);
+ coderArray.get(idx).encode(value.getFieldValue(idx), outStream);
}
instantCoder.encode(value.getWindowStart(), outStream);
@@ -70,7 +74,6 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
BitSet nullFields = nullListCoder.decode(inStream);
BeamRecord record = new BeamRecord(recordType);
- record.setNullFields(nullFields);
for (int idx = 0; idx < recordType.size(); ++idx) {
if (nullFields.get(idx)) {
continue;
@@ -88,5 +91,8 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
@Override
public void verifyDeterministic()
throws org.apache.beam.sdk.coders.Coder.NonDeterministicException {
+ for (Coder c : coderArray) {
+ c.verifyDeterministic();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
index 476233e..bac649e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecord.java
@@ -32,7 +32,7 @@ import org.joda.time.Instant;
/**
* {@link org.apache.beam.sdk.values.BeamRecord}, self-described with
- * {@link BeamRecordTypeProvider}, represents one element in a
+ * {@link BeamRecordType}, represents one element in a
* {@link org.apache.beam.sdk.values.PCollection}.
*/
@Experimental
@@ -40,12 +40,12 @@ public class BeamRecord implements Serializable {
private List<Object> dataValues;
//null values are indexed here, to handle properly in Coder.
private BitSet nullFields;
- private BeamRecordTypeProvider dataType;
+ private BeamRecordType dataType;
private Instant windowStart = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MIN_VALUE));
private Instant windowEnd = new Instant(TimeUnit.MICROSECONDS.toMillis(Long.MAX_VALUE));
- public BeamRecord(BeamRecordTypeProvider dataType) {
+ public BeamRecord(BeamRecordType dataType) {
this.dataType = dataType;
this.nullFields = new BitSet(dataType.size());
this.dataValues = new ArrayList<>();
@@ -55,7 +55,7 @@ public class BeamRecord implements Serializable {
}
}
- public BeamRecord(BeamRecordTypeProvider dataType, List<Object> dataValues) {
+ public BeamRecord(BeamRecordType dataType, List<Object> dataValues) {
this(dataType);
for (int idx = 0; idx < dataValues.size(); ++idx) {
addField(idx, dataValues.get(idx));
@@ -137,10 +137,6 @@ public class BeamRecord implements Serializable {
}
public Object getFieldValue(int fieldIdx) {
- if (nullFields.get(fieldIdx)) {
- return null;
- }
-
return dataValues.get(fieldIdx);
}
@@ -200,22 +196,14 @@ public class BeamRecord implements Serializable {
this.dataValues = dataValues;
}
- public BeamRecordTypeProvider getDataType() {
+ public BeamRecordType getDataType() {
return dataType;
}
- public void setDataType(BeamRecordTypeProvider dataType) {
- this.dataType = dataType;
- }
-
public BitSet getNullFields() {
return nullFields;
}
- public void setNullFields(BitSet nullFields) {
- this.nullFields = nullFields;
- }
-
/**
* is the specified field NULL?
*/
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordType.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordType.java
new file mode 100644
index 0000000..3b20b50
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordType.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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.values;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
+import org.apache.beam.sdk.coders.Coder;
+
+/**
+ * The default type provider used in {@link BeamRecord}.
+ */
+@Experimental
+public class BeamRecordType implements Serializable{
+ private List<String> fieldsName;
+ private List<Coder> fieldsCoder;
+
+ public BeamRecordType(List<String> fieldsName, List<Coder> fieldsCoder) {
+ this.fieldsName = fieldsName;
+ this.fieldsCoder = fieldsCoder;
+ }
+
+ /**
+ * Validate input fieldValue for a field.
+ * @throws IllegalArgumentException throw exception when the validation fails.
+ */
+ public void validateValueType(int index, Object fieldValue)
+ throws IllegalArgumentException{
+ //do nothing by default.
+ }
+
+ /**
+ * Get the coder for {@link BeamRecordCoder}.
+ */
+ public BeamRecordCoder getRecordCoder(){
+ return BeamRecordCoder.of(this, fieldsCoder);
+ }
+
+ public List<String> getFieldsName(){
+ return fieldsName;
+ }
+
+ public String getFieldByIndex(int index){
+ return fieldsName.get(index);
+ }
+
+ public int findIndexOfField(String fieldName){
+ return fieldsName.indexOf(fieldName);
+ }
+
+ public int size(){
+ return fieldsName.size();
+ }
+}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.java
deleted file mode 100644
index 63a961c..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/BeamRecordTypeProvider.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.beam.sdk.values;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.beam.sdk.annotations.Experimental;
-
-/**
- * The default type provider used in {@link BeamRecord}.
- */
-@Experimental
-public class BeamRecordTypeProvider implements Serializable{
- private List<String> fieldsName;
-
- public BeamRecordTypeProvider(List<String> fieldsName) {
- this.fieldsName = fieldsName;
- }
-
- /**
- * Validate input fieldValue for a field.
- * @throws IllegalArgumentException throw exception when the validation fails.
- */
- public void validateValueType(int index, Object fieldValue)
- throws IllegalArgumentException{
- //do nothing by default.
- }
-
- public List<String> getFieldsName(){
- return fieldsName;
- }
-
- public String getFieldByIndex(int index){
- return fieldsName.get(index);
- }
-
- public int findIndexOfField(String fieldName){
- return fieldsName.indexOf(fieldName);
- }
-
- public int size(){
- return fieldsName.size();
- }
-}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
index 0dabf40..86e4d8d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSql.java
@@ -19,13 +19,14 @@ package org.apache.beam.sdk.extensions.sql;
import com.google.auto.value.AutoValue;
import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.BeamRecordCoder;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.extensions.sql.schema.BeamPCollectionTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TupleTag;
@@ -111,7 +112,7 @@ public class BeamSql {
*/
@AutoValue
public abstract static class QueryTransform extends
- PTransform<PCollectionTuple, PCollection<BeamSqlRow>> {
+ PTransform<PCollectionTuple, PCollection<BeamRecord>> {
abstract BeamSqlEnv getSqlEnv();
abstract String getSqlQuery();
@@ -143,7 +144,7 @@ public class BeamSql {
}
@Override
- public PCollection<BeamSqlRow> expand(PCollectionTuple input) {
+ public PCollection<BeamRecord> expand(PCollectionTuple input) {
registerTables(input);
BeamRelNode beamRelNode = null;
@@ -163,11 +164,12 @@ public class BeamSql {
//register tables, related with input PCollections.
private void registerTables(PCollectionTuple input){
for (TupleTag<?> sourceTag : input.getAll().keySet()) {
- PCollection<BeamSqlRow> sourceStream = (PCollection<BeamSqlRow>) input.get(sourceTag);
- BeamSqlRowCoder sourceCoder = (BeamSqlRowCoder) sourceStream.getCoder();
+ PCollection<BeamRecord> sourceStream = (PCollection<BeamRecord>) input.get(sourceTag);
+ BeamRecordCoder sourceCoder = (BeamRecordCoder) sourceStream.getCoder();
getSqlEnv().registerTable(sourceTag.getId(),
- new BeamPCollectionTable(sourceStream, sourceCoder.getSqlRecordType()));
+ new BeamPCollectionTable(sourceStream,
+ (BeamSqlRecordType) sourceCoder.getRecordType()));
}
}
}
@@ -178,7 +180,7 @@ public class BeamSql {
*/
@AutoValue
public abstract static class SimpleQueryTransform
- extends PTransform<PCollection<BeamSqlRow>, PCollection<BeamSqlRow>> {
+ extends PTransform<PCollection<BeamRecord>, PCollection<BeamRecord>> {
private static final String PCOLLECTION_TABLE_NAME = "PCOLLECTION";
abstract BeamSqlEnv getSqlEnv();
abstract String getSqlQuery();
@@ -232,9 +234,9 @@ public class BeamSql {
}
@Override
- public PCollection<BeamSqlRow> expand(PCollection<BeamSqlRow> input) {
+ public PCollection<BeamRecord> expand(PCollection<BeamRecord> input) {
validateQuery();
- return PCollectionTuple.of(new TupleTag<BeamSqlRow>(PCOLLECTION_TABLE_NAME), input)
+ return PCollectionTuple.of(new TupleTag<BeamRecord>(PCOLLECTION_TABLE_NAME), input)
.apply(QueryTransform.builder()
.setSqlEnv(getSqlEnv())
.setSqlQuery(getSqlQuery())
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
index 967dee5..a43808e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -20,9 +20,9 @@ package org.apache.beam.sdk.extensions.sql;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PCollection;
import org.apache.calcite.plan.RelOptUtil;
@@ -43,7 +43,7 @@ public class BeamSqlCli {
/**
* compile SQL, and return a {@link Pipeline}.
*/
- public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
+ public static PCollection<BeamRecord> compilePipeline(String sqlStatement, BeamSqlEnv sqlEnv)
throws Exception{
PipelineOptions options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation()
.as(PipelineOptions.class); // FlinkPipelineOptions.class
@@ -56,9 +56,9 @@ public class BeamSqlCli {
/**
* compile SQL, and return a {@link Pipeline}.
*/
- public static PCollection<BeamSqlRow> compilePipeline(String sqlStatement, Pipeline basePipeline,
+ public static PCollection<BeamRecord> compilePipeline(String sqlStatement, Pipeline basePipeline,
BeamSqlEnv sqlEnv) throws Exception{
- PCollection<BeamSqlRow> resultStream =
+ PCollection<BeamRecord> resultStream =
sqlEnv.planner.compileBeamPipeline(sqlStatement, basePipeline, sqlEnv);
return resultStream;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
index be0b0af..3c5eb36 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlEnv.java
@@ -21,7 +21,7 @@ import java.io.Serializable;
import org.apache.beam.sdk.extensions.sql.impl.planner.BeamQueryPlanner;
import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
import org.apache.beam.sdk.extensions.sql.schema.BaseBeamTable;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdaf;
import org.apache.beam.sdk.extensions.sql.schema.BeamSqlUdf;
import org.apache.calcite.DataContext;
@@ -84,8 +84,8 @@ public class BeamSqlEnv implements Serializable{
}
private static class BeamCalciteTable implements ScannableTable, Serializable {
- private BeamSqlRowType beamSqlRowType;
- public BeamCalciteTable(BeamSqlRowType beamSqlRowType) {
+ private BeamSqlRecordType beamSqlRowType;
+ public BeamCalciteTable(BeamSqlRecordType beamSqlRowType) {
this.beamSqlRowType = beamSqlRowType;
}
@Override
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
index 21e02a7..fbc1fd8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlExample.java
@@ -22,14 +22,13 @@ import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.extensions.sql.BeamSql;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowCoder;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRowType;
+import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRecordType;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
@@ -54,39 +53,39 @@ class BeamSqlExample {
//define the input row format
List<String> fieldNames = Arrays.asList("c1", "c2", "c3");
List<Integer> fieldTypes = Arrays.asList(Types.INTEGER, Types.VARCHAR, Types.DOUBLE);
- BeamSqlRowType type = BeamSqlRowType.create(fieldNames, fieldTypes);
- BeamSqlRow row = new BeamSqlRow(type);
+ BeamSqlRecordType type = BeamSqlRecordType.create(fieldNames, fieldTypes);
+ BeamRecord row = new BeamRecord(type);
row.addField(0, 1);
row.addField(1, "row");
row.addField(2, 1.0);
//create a source PCollection with Create.of();
- PCollection<BeamSqlRow> inputTable = PBegin.in(p).apply(Create.of(row)
- .withCoder(new BeamSqlRowCoder(type)));
+ PCollection<BeamRecord> inputTable = PBegin.in(p).apply(Create.of(row)
+ .withCoder(type.getRecordCoder()));
//Case 1. run a simple SQL query over input PCollection with BeamSql.simpleQuery;
- PCollection<BeamSqlRow> outputStream = inputTable.apply(
+ PCollection<BeamRecord> outputStream = inputTable.apply(
BeamSql.simpleQuery("select c1, c2, c3 from PCOLLECTION where c1=1"));
//print the output record of case 1;
outputStream.apply("log_result",
- MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
- public Void apply(BeamSqlRow input) {
+ MapElements.<BeamRecord, Void>via(new SimpleFunction<BeamRecord, Void>() {
+ public Void apply(BeamRecord input) {
System.out.println("PCOLLECTION: " + input);
return null;
}
}));
//Case 2. run the query with BeamSql.query over result PCollection of case 1.
- PCollection<BeamSqlRow> outputStream2 =
- PCollectionTuple.of(new TupleTag<BeamSqlRow>("CASE1_RESULT"), outputStream)
+ PCollection<BeamRecord> outputStream2 =
+ PCollectionTuple.of(new TupleTag<BeamRecord>("CASE1_RESULT"), outputStream)
.apply(BeamSql.query("select c2, c3 from CASE1_RESULT where c1=1"));
//print the output record of case 2;
outputStream2.apply("log_result",
- MapElements.<BeamSqlRow, Void>via(new SimpleFunction<BeamSqlRow, Void>() {
+ MapElements.<BeamRecord, Void>via(new SimpleFunction<BeamRecord, Void>() {
@Override
- public Void apply(BeamSqlRow input) {
+ public Void apply(BeamRecord input) {
System.out.println("TABLE_B: " + input);
return null;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
index 1ae6bb3..3cd6d65 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlExpressionExecutor.java
@@ -19,7 +19,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter;
import java.io.Serializable;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
/**
* {@code BeamSqlExpressionExecutor} fills the gap between relational
@@ -34,10 +34,10 @@ public interface BeamSqlExpressionExecutor extends Serializable {
void prepare();
/**
- * apply transformation to input record {@link BeamSqlRow}.
+ * apply transformation to input record {@link BeamRecord}.
*
*/
- List<Object> execute(BeamSqlRow inputRow);
+ List<Object> execute(BeamRecord inputRow);
void close();
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
index 1f9e0e3..0f77ed8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/BeamSqlFnExecutor.java
@@ -88,7 +88,7 @@ import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string.BeamS
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamFilterRel;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamProjectRel;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexInputRef;
import org.apache.calcite.rex.RexLiteral;
@@ -102,7 +102,7 @@ import org.apache.calcite.util.NlsString;
/**
* Executor based on {@link BeamSqlExpression} and {@link BeamSqlPrimitive}.
* {@code BeamSqlFnExecutor} converts a {@link BeamRelNode} to a {@link BeamSqlExpression},
- * which can be evaluated against the {@link BeamSqlRow}.
+ * which can be evaluated against the {@link BeamRecord}.
*
*/
public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
@@ -427,7 +427,7 @@ public class BeamSqlFnExecutor implements BeamSqlExpressionExecutor {
}
@Override
- public List<Object> execute(BeamSqlRow inputRow) {
+ public List<Object> execute(BeamRecord inputRow) {
List<Object> results = new ArrayList<>();
for (BeamSqlExpression exp : exps) {
results.add(exp.evaluate(inputRow).getValue());
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
index 61e8aae..af48cbe 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCaseExpression.java
@@ -19,7 +19,7 @@
package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -48,7 +48,7 @@ public class BeamSqlCaseExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
for (int i = 0; i < operands.size() - 1; i += 2) {
if (opValueEvaluated(i, inputRow)) {
return BeamSqlPrimitive.of(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
index c98c10d..3786281 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlCastExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.sql.Date;
import java.sql.Timestamp;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.runtime.SqlFunctions;
import org.apache.calcite.sql.type.SqlTypeName;
import org.joda.time.format.DateTimeFormat;
@@ -71,7 +71,7 @@ public class BeamSqlCastExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
SqlTypeName castOutputType = getOutputType();
switch (castOutputType) {
case INTEGER:
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
index dc5db81..f42a365 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlExpression.java
@@ -19,7 +19,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.io.Serializable;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -49,7 +49,7 @@ public abstract class BeamSqlExpression implements Serializable {
return op(idx).getOutputType();
}
- public <T> T opValueEvaluated(int idx, BeamSqlRow row) {
+ public <T> T opValueEvaluated(int idx, BeamRecord row) {
return (T) op(idx).evaluate(row).getValue();
}
@@ -59,10 +59,10 @@ public abstract class BeamSqlExpression implements Serializable {
public abstract boolean accept();
/**
- * Apply input record {@link BeamSqlRow} to this expression,
+ * Apply input record {@link BeamRecord} to this expression,
* the output value is wrapped with {@link BeamSqlPrimitive}.
*/
- public abstract BeamSqlPrimitive evaluate(BeamSqlRow inputRow);
+ public abstract BeamSqlPrimitive evaluate(BeamRecord inputRow);
public List<BeamSqlExpression> getOperands() {
return operands;
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
index 7aba024..8c3d4d4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlInputRefExpression.java
@@ -17,7 +17,7 @@
*/
package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -37,7 +37,7 @@ public class BeamSqlInputRefExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(outputType, inputRow.getFieldValue(inputRef));
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
index 6380af9..f763898 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlPrimitive.java
@@ -21,13 +21,13 @@ import java.math.BigDecimal;
import java.util.Date;
import java.util.GregorianCalendar;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.NlsString;
/**
* {@link BeamSqlPrimitive} is a special, self-reference {@link BeamSqlExpression}.
- * It holds the value, and return it directly during {@link #evaluate(BeamSqlRow)}.
+ * It holds the value, and return it directly during {@link #evaluate(BeamRecord)}.
*
*/
public class BeamSqlPrimitive<T> extends BeamSqlExpression {
@@ -145,7 +145,7 @@ public class BeamSqlPrimitive<T> extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<T> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<T> evaluate(BeamRecord inputRow) {
return this;
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
index 243baaa..c1fa2c7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlReinterpretExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.util.Date;
import java.util.GregorianCalendar;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -41,7 +41,7 @@ public class BeamSqlReinterpretExpression extends BeamSqlExpression {
&& SqlTypeName.DATETIME_TYPES.contains(opType(0));
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
if (opType(0) == SqlTypeName.TIME) {
GregorianCalendar date = opValueEvaluated(0, inputRow);
return BeamSqlPrimitive.of(outputType, date.getTimeInMillis());
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
index eebb97c..da706f3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlUdfExpression.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -51,7 +51,7 @@ public class BeamSqlUdfExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
if (method == null) {
reConstructMethod();
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
index 0bd68df..2f4c165 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowEndExpression.java
@@ -18,7 +18,7 @@
package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -34,7 +34,7 @@ public class BeamSqlWindowEndExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Date> evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
new Date(inputRow.getWindowEnd().getMillis()));
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
index b560ef8..2f3dd5c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowExpression.java
@@ -19,7 +19,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.util.Date;
import java.util.List;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -42,7 +42,7 @@ public class BeamSqlWindowExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Date> evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
(Date) operands.get(0).evaluate(inputRow).getValue());
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
index e2c1b34..9186ec0 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/BeamSqlWindowStartExpression.java
@@ -18,7 +18,7 @@
package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
import java.util.Date;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -35,7 +35,7 @@ public class BeamSqlWindowStartExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Date> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Date> evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(SqlTypeName.TIMESTAMP,
new Date(inputRow.getWindowStart().getMillis()));
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
index b07b28f..fd36457 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/arithmetic/BeamSqlArithmeticExpression.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -50,7 +50,7 @@ public abstract class BeamSqlArithmeticExpression extends BeamSqlExpression {
super(operands, outputType);
}
- @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamRecord inputRow) {
BigDecimal left = BigDecimal.valueOf(
Double.valueOf(opValueEvaluated(0, inputRow).toString()));
BigDecimal right = BigDecimal.valueOf(
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
index 811b582..93032ae 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlCompareExpression.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.comparison;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -51,7 +51,7 @@ public abstract class BeamSqlCompareExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Boolean> evaluate(BeamRecord inputRow) {
Object leftValue = operands.get(0).evaluate(inputRow).getValue();
Object rightValue = operands.get(1).evaluate(inputRow).getValue();
switch (operands.get(0).getOutputType()) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
index 88dc73f..7177d96 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNotNullExpression.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -46,7 +46,7 @@ public class BeamSqlIsNotNullExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Boolean> evaluate(BeamRecord inputRow) {
Object leftValue = operands.get(0).evaluate(inputRow).getValue();
return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue != null);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
index b626ce7..c74fcd9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/comparison/BeamSqlIsNullExpression.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -46,7 +46,7 @@ public class BeamSqlIsNullExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Boolean> evaluate(BeamRecord inputRow) {
Object leftValue = operands.get(0).evaluate(inputRow).getValue();
return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, leftValue == null);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
index d5793d5..86abe43 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentDateExpression.java
@@ -22,7 +22,7 @@ import java.util.Collections;
import java.util.Date;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -38,7 +38,7 @@ public class BeamSqlCurrentDateExpression extends BeamSqlExpression {
return getOperands().size() == 0;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(outputType, new Date());
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
index 99eea95..d8de464 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimeExpression.java
@@ -24,7 +24,7 @@ import java.util.List;
import java.util.TimeZone;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -44,7 +44,7 @@ public class BeamSqlCurrentTimeExpression extends BeamSqlExpression {
return opCount <= 1;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
GregorianCalendar ret = new GregorianCalendar(TimeZone.getDefault());
ret.setTime(new Date());
return BeamSqlPrimitive.of(outputType, ret);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
index 09a3c60..4736571 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlCurrentTimestampExpression.java
@@ -22,7 +22,7 @@ import java.util.Date;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -42,7 +42,7 @@ public class BeamSqlCurrentTimestampExpression extends BeamSqlExpression {
return opCount <= 1;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(outputType, new Date());
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
index 55b6fcd..55767fa 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateCeilExpression.java
@@ -22,7 +22,7 @@ import java.util.Date;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.avatica.util.DateTimeUtils;
import org.apache.calcite.avatica.util.TimeUnitRange;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -41,7 +41,7 @@ public class BeamSqlDateCeilExpression extends BeamSqlExpression {
&& opType(1) == SqlTypeName.SYMBOL;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
Date date = opValueEvaluated(0, inputRow);
long time = date.getTime();
TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
index f031c31..3310da5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlDateFloorExpression.java
@@ -22,7 +22,7 @@ import java.util.Date;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.avatica.util.DateTimeUtils;
import org.apache.calcite.avatica.util.TimeUnitRange;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -41,7 +41,7 @@ public class BeamSqlDateFloorExpression extends BeamSqlExpression {
&& opType(1) == SqlTypeName.SYMBOL;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
Date date = opValueEvaluated(0, inputRow);
long time = date.getTime();
TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(1)).getValue();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
index 2740f82..47cd879 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/date/BeamSqlExtractExpression.java
@@ -25,7 +25,7 @@ import java.util.List;
import java.util.Map;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.avatica.util.DateTimeUtils;
import org.apache.calcite.avatica.util.TimeUnitRange;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -61,7 +61,7 @@ public class BeamSqlExtractExpression extends BeamSqlExpression {
&& opType(1) == SqlTypeName.BIGINT;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
Long time = opValueEvaluated(1, inputRow);
TimeUnitRange unit = ((BeamSqlPrimitive<TimeUnitRange>) op(0)).getValue();
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
index 0c8854c..b8964d5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlAndExpression.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlAndExpression extends BeamSqlLogicalExpression {
}
@Override
- public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Boolean> evaluate(BeamRecord inputRow) {
boolean result = true;
for (BeamSqlExpression exp : operands) {
BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
index 65634b0..f9578b9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlNotExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -42,7 +42,7 @@ public class BeamSqlNotExpression extends BeamSqlLogicalExpression {
return super.accept();
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
Boolean value = opValueEvaluated(0, inputRow);
if (value == null) {
return BeamSqlPrimitive.of(SqlTypeName.BOOLEAN, null);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
index da15c34..88a3916 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/logical/BeamSqlOrExpression.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.logical;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlOrExpression extends BeamSqlLogicalExpression {
}
@Override
- public BeamSqlPrimitive<Boolean> evaluate(BeamSqlRow inputRow) {
+ public BeamSqlPrimitive<Boolean> evaluate(BeamRecord inputRow) {
boolean result = false;
for (BeamSqlExpression exp : operands) {
BeamSqlPrimitive<Boolean> expOut = exp.evaluate(inputRow);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
index c12b725..8f6c00c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathBinaryExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -38,7 +38,7 @@ public abstract class BeamSqlMathBinaryExpression extends BeamSqlExpression {
return numberOfOperands() == 2 && isOperandNumeric(opType(0)) && isOperandNumeric(opType(1));
}
- @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamRecord inputRow) {
BeamSqlExpression leftOp = op(0);
BeamSqlExpression rightOp = op(1);
return calculate(leftOp.evaluate(inputRow), rightOp.evaluate(inputRow));
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
index 163c40e..b225b8e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlMathUnaryExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
@@ -45,7 +45,7 @@ public abstract class BeamSqlMathUnaryExpression extends BeamSqlExpression {
return acceptance;
}
- @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive<? extends Number> evaluate(BeamRecord inputRow) {
BeamSqlExpression operand = op(0);
return calculate(operand.evaluate(inputRow));
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
index dfaf546..676f859 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlPiExpression.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.math;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -36,7 +36,7 @@ public class BeamSqlPiExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
return BeamSqlPrimitive.of(SqlTypeName.DOUBLE, Math.PI);
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
index f2d7a47..0575978 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandExpression.java
@@ -22,7 +22,7 @@ import java.util.List;
import java.util.Random;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -42,7 +42,7 @@ public class BeamSqlRandExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+ public BeamSqlPrimitive evaluate(BeamRecord inputRecord) {
if (operands.size() == 1) {
int rowSeed = opValueEvaluated(0, inputRecord);
if (seed == null || seed != rowSeed) {
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
index b2e65ce..52f0cc1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/math/BeamSqlRandIntegerExpression.java
@@ -22,7 +22,7 @@ import java.util.List;
import java.util.Random;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -43,7 +43,7 @@ public class BeamSqlRandIntegerExpression extends BeamSqlExpression {
}
@Override
- public BeamSqlPrimitive evaluate(BeamSqlRow inputRecord) {
+ public BeamSqlPrimitive evaluate(BeamRecord inputRecord) {
int numericIdx = 0;
if (operands.size() == 2) {
int rowSeed = opValueEvaluated(0, inputRecord);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
index 580d747..974e2bc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlCharLengthExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlCharLengthExpression extends BeamSqlStringUnaryExpression {
super(operands, SqlTypeName.INTEGER);
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
return BeamSqlPrimitive.of(SqlTypeName.INTEGER, str.length());
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
index 772ad41..14ef55d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlConcatExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -51,7 +51,7 @@ public class BeamSqlConcatExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String left = opValueEvaluated(0, inputRow);
String right = opValueEvaluated(1, inputRow);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
index dc893e7..e50872b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlInitCapExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlInitCapExpression extends BeamSqlStringUnaryExpression {
super(operands, SqlTypeName.VARCHAR);
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
StringBuilder ret = new StringBuilder(str);
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
index fd9d7aa..0f9a501 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlLowerExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -32,7 +32,7 @@ public class BeamSqlLowerExpression extends BeamSqlStringUnaryExpression {
super(operands, SqlTypeName.VARCHAR);
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, str.toLowerCase());
}
http://git-wip-us.apache.org/repos/asf/beam/blob/89109b8c/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
index 8d38efb..2336876 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/string/BeamSqlOverlayExpression.java
@@ -21,7 +21,7 @@ package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.string;
import java.util.List;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlExpression;
import org.apache.beam.sdk.extensions.sql.impl.interpreter.operator.BeamSqlPrimitive;
-import org.apache.beam.sdk.extensions.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.values.BeamRecord;
import org.apache.calcite.sql.type.SqlTypeName;
/**
@@ -54,7 +54,7 @@ public class BeamSqlOverlayExpression extends BeamSqlExpression {
return true;
}
- @Override public BeamSqlPrimitive evaluate(BeamSqlRow inputRow) {
+ @Override public BeamSqlPrimitive evaluate(BeamRecord inputRow) {
String str = opValueEvaluated(0, inputRow);
String replaceStr = opValueEvaluated(1, inputRow);
int idx = opValueEvaluated(2, inputRow);