This is an automated email from the ASF dual-hosted git repository.
mcasters pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hop.git
The following commit(s) were added to refs/heads/master by this push:
new 20f5baabbe Handle timestamp values in BigQueryOutputTransform
new 654baab72f Merge pull request #1719 from fbarthez/HOP-4199
20f5baabbe is described below
commit 20f5baabbe479e3d6d62de551a6c59e6543a6eb5
Author: Fabian Peters <[email protected]>
AuthorDate: Tue Oct 4 15:07:28 2022 +0200
Handle timestamp values in BigQueryOutputTransform
---
.../main/java/org/apache/hop/beam/core/fn/HopToBQTableRowFn.java | 8 ++++++++
.../org/apache/hop/beam/core/transform/BeamBQOutputTransform.java | 3 +++
2 files changed, 11 insertions(+)
diff --git
a/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/fn/HopToBQTableRowFn.java
b/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/fn/HopToBQTableRowFn.java
index ca1865d626..0601719211 100644
---
a/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/fn/HopToBQTableRowFn.java
+++
b/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/fn/HopToBQTableRowFn.java
@@ -106,6 +106,14 @@ public class HopToBQTableRowFn implements
SerializableFunction<HopRow, TableRow>
case IValueMeta.TYPE_NUMBER:
tableRow.put(valueMeta.getName(),
valueMeta.getNumber(valueData));
break;
+ case IValueMeta.TYPE_TIMESTAMP:
+ if (valueMeta.getNumber(valueData) != null) {
+ // we have a nanoseconds value and BigQuery expects
microseconds, so divide by 1000
+ tableRow.put(valueMeta.getName(),
Math.round(valueMeta.getNumber(valueData) / 1000));
+ } else {
+ tableRow.put(valueMeta.getName(),
valueMeta.getNumber(valueData));
+ }
+ break;
default:
throw new RuntimeException(
"Data type conversion from Hop to BigQuery TableRow not
supported yet: "
diff --git
a/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/transform/BeamBQOutputTransform.java
b/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/transform/BeamBQOutputTransform.java
index 9ebab29c09..efa8396792 100644
---
a/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/transform/BeamBQOutputTransform.java
+++
b/plugins/engines/beam/src/main/java/org/apache/hop/beam/core/transform/BeamBQOutputTransform.java
@@ -130,6 +130,9 @@ public class BeamBQOutputTransform extends
PTransform<PCollection<HopRow>, PDone
case IValueMeta.TYPE_NUMBER:
schemaField.setType("FLOAT");
break;
+ case IValueMeta.TYPE_TIMESTAMP:
+ schemaField.setType("TIMESTAMP");
+ break;
default:
throw new RuntimeException(
"Conversion from Hop value "