You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ta...@apache.org on 2017/08/08 18:21:08 UTC
[1/2] beam git commit: [BEAM-2742] change Field type from primitives
to boxed types
Repository: beam
Updated Branches:
refs/heads/DSL_SQL d7120f077 -> 926c70a34
[BEAM-2742] change Field type from primitives to boxed types
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/990fedc9
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/990fedc9
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/990fedc9
Branch: refs/heads/DSL_SQL
Commit: 990fedc934da4eee2ec34c6a9ed5bd5a2dacbded
Parents: d7120f0
Author: James Xu <xu...@gmail.com>
Authored: Tue Aug 8 14:26:09 2017 +0800
Committer: Tyler Akidau <ta...@apache.org>
Committed: Tue Aug 8 11:17:46 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 4 +-
.../org/apache/beam/sdk/values/BeamRecord.java | 39 ++++++++------------
.../extensions/sql/impl/rel/BeamSortRel.java | 8 ++--
3 files changed, 23 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/beam/blob/990fedc9/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 40b9f3f..a6200f6 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
@@ -57,7 +57,7 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
throws CoderException, IOException {
nullListCoder.encode(scanNullFields(value), outStream);
for (int idx = 0; idx < value.size(); ++idx) {
- if (value.isNull(idx)) {
+ if (value.getFieldValue(idx) == null) {
continue;
}
@@ -87,7 +87,7 @@ public class BeamRecordCoder extends CustomCoder<BeamRecord> {
private BitSet scanNullFields(BeamRecord record){
BitSet nullFields = new BitSet(record.size());
for (int idx = 0; idx < record.size(); ++idx) {
- if (record.isNull(idx)) {
+ if (record.getFieldValue(idx) == null) {
nullFields.set(idx);
}
}
http://git-wip-us.apache.org/repos/asf/beam/blob/990fedc9/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 6cbd11b..35a96f6 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
@@ -63,27 +63,27 @@ public class BeamRecord implements Serializable {
return getFieldValue(dataType.getFieldsName().indexOf(fieldName));
}
- public byte getByte(String fieldName) {
+ public Byte getByte(String fieldName) {
return (Byte) getFieldValue(fieldName);
}
- public short getShort(String fieldName) {
+ public Short getShort(String fieldName) {
return (Short) getFieldValue(fieldName);
}
- public int getInteger(String fieldName) {
+ public Integer getInteger(String fieldName) {
return (Integer) getFieldValue(fieldName);
}
- public float getFloat(String fieldName) {
+ public Float getFloat(String fieldName) {
return (Float) getFieldValue(fieldName);
}
- public double getDouble(String fieldName) {
+ public Double getDouble(String fieldName) {
return (Double) getFieldValue(fieldName);
}
- public long getLong(String fieldName) {
+ public Long getLong(String fieldName) {
return (Long) getFieldValue(fieldName);
}
@@ -103,35 +103,35 @@ public class BeamRecord implements Serializable {
return (BigDecimal) getFieldValue(fieldName);
}
- public boolean getBoolean(String fieldName) {
- return (boolean) getFieldValue(fieldName);
+ public Boolean getBoolean(String fieldName) {
+ return (Boolean) getFieldValue(fieldName);
}
public Object getFieldValue(int fieldIdx) {
return dataValues.get(fieldIdx);
}
- public byte getByte(int idx) {
+ public Byte getByte(int idx) {
return (Byte) getFieldValue(idx);
}
- public short getShort(int idx) {
+ public Short getShort(int idx) {
return (Short) getFieldValue(idx);
}
- public int getInteger(int idx) {
+ public Integer getInteger(int idx) {
return (Integer) getFieldValue(idx);
}
- public float getFloat(int idx) {
+ public Float getFloat(int idx) {
return (Float) getFieldValue(idx);
}
- public double getDouble(int idx) {
+ public Double getDouble(int idx) {
return (Double) getFieldValue(idx);
}
- public long getLong(int idx) {
+ public Long getLong(int idx) {
return (Long) getFieldValue(idx);
}
@@ -151,8 +151,8 @@ public class BeamRecord implements Serializable {
return (BigDecimal) getFieldValue(idx);
}
- public boolean getBoolean(int idx) {
- return (boolean) getFieldValue(idx);
+ public Boolean getBoolean(int idx) {
+ return (Boolean) getFieldValue(idx);
}
public int size() {
@@ -171,13 +171,6 @@ public class BeamRecord implements Serializable {
return dataType;
}
- /**
- * is the specified field NULL?
- */
- public boolean isNull(int idx) {
- return null == getFieldValue(idx);
- }
-
@Override
public String toString() {
return "BeamSqlRow [dataValues=" + dataValues + ", dataType=" + dataType + "]";
http://git-wip-us.apache.org/repos/asf/beam/blob/990fedc9/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 e98ead1..80f3c97 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
@@ -195,11 +195,13 @@ public class BeamSortRel extends Sort implements BeamRelNode {
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)) {
+ boolean isValue1Null = (row1.getFieldValue(fieldIndex) == null);
+ boolean isValue2Null = (row2.getFieldValue(fieldIndex) == null);
+ if (isValue1Null && isValue2Null) {
continue;
- } else if (row1.isNull(fieldIndex) && !row2.isNull(fieldIndex)) {
+ } else if (isValue1Null && !isValue2Null) {
fieldRet = -1 * (nullsFirst.get(i) ? -1 : 1);
- } else if (!row1.isNull(fieldIndex) && row2.isNull(fieldIndex)) {
+ } else if (!isValue1Null && isValue2Null) {
fieldRet = 1 * (nullsFirst.get(i) ? -1 : 1);
} else {
switch (fieldType) {
[2/2] beam git commit: [BEAM-2742] This closes #3699
Posted by ta...@apache.org.
[BEAM-2742] This closes #3699
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/926c70a3
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/926c70a3
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/926c70a3
Branch: refs/heads/DSL_SQL
Commit: 926c70a345912b084fbb9366e9ecf2ecc4450739
Parents: d7120f0 990fedc
Author: Tyler Akidau <ta...@apache.org>
Authored: Tue Aug 8 11:18:49 2017 -0700
Committer: Tyler Akidau <ta...@apache.org>
Committed: Tue Aug 8 11:18:49 2017 -0700
----------------------------------------------------------------------
.../apache/beam/sdk/coders/BeamRecordCoder.java | 4 +-
.../org/apache/beam/sdk/values/BeamRecord.java | 39 ++++++++------------
.../extensions/sql/impl/rel/BeamSortRel.java | 8 ++--
3 files changed, 23 insertions(+), 28 deletions(-)
----------------------------------------------------------------------