[GitHub] [nifi] pvillard31 commented on a diff in pull request #7489: NIFI-11823 - fix NUMERIC support in PutBigQuery

2023-08-02 Thread via GitHub


pvillard31 commented on code in PR #7489:
URL: https://github.com/apache/nifi/pull/7489#discussion_r1282394555


##
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/proto/ProtoUtils.java:
##
@@ -17,50 +17,99 @@
 
 package org.apache.nifi.processors.gcp.bigquery.proto;
 
+import com.google.cloud.bigquery.storage.v1.BigDecimalByteStringEncoder;
+import com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type;
+import com.google.cloud.bigquery.storage.v1.TableSchema;
 import com.google.protobuf.Descriptors;
+import com.google.protobuf.DoubleValue;
 import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int64Value;
+
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
 
 /**
- * Util class for protocol buffer messaging
- */
+* Util class for protocol buffer messaging
+*/
 public class ProtoUtils {
 
-public static DynamicMessage createMessage(Descriptors.Descriptor 
descriptor, Map valueMap) {
-DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor);
+   public static DynamicMessage createMessage(Descriptors.Descriptor 
descriptor, Map valueMap, TableSchema tableSchema) {
+   final DynamicMessage.Builder builder = 
DynamicMessage.newBuilder(descriptor);
+
+   for (final Descriptors.FieldDescriptor field : descriptor.getFields()) {
+   final String name = field.getName();
+   Object value = valueMap.get(name);
+   if (value == null) {
+   continue;
+   }
+
+   switch (field.getType()) {
+   case MESSAGE:
+   if (field.isRepeated()) {
+   Collection collection = value.getClass().isArray() ? 
Arrays.asList((Object[]) value) : (Collection) value;
+   collection.forEach(act -> builder.addRepeatedField(field, 
createMessage(field.getMessageType(), (Map) act, tableSchema)));
+   } else {
+   builder.setField(field, 
createMessage(field.getMessageType(), (Map) value, 
tableSchema));
+   }
+   break;
+
+   // INT64 with alias INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT
+   case INT64:
+   // Integer in the bigquery table schema maps back to INT64 
which is considered to be Long on Java side:
+   // https://developers.google.com/protocol-buffers/docs/proto3
+   if (value instanceof Integer) {
+   value = Long.valueOf((Integer) value);
+   }
+
+   setField(value, field, builder);
+   break;
+
+   // FLOAT64
+   case DOUBLE:
+   if (value instanceof Float) {
+   value = ((Float) value).doubleValue();
+   }
+   setField(value, field, builder);
+   break;
+
+   // matches NUMERIC and BIGNUMERIC types in BigQuery
+   // BQTableSchemaToProtoDescriptor.class
+   case BYTES:
+   if(value instanceof BigDecimal) {
+   
if(tableSchema.getFields(field.getIndex()).getType().equals(Type.BIGNUMERIC)) {
+   value = 
BigDecimalByteStringEncoder.encodeToBigNumericByteString((BigDecimal) value);
+   } else if 
(tableSchema.getFields(field.getIndex()).getType().equals(Type.NUMERIC)) {
+   value = 
BigDecimalByteStringEncoder.encodeToNumericByteString((BigDecimal) value);
+   }
+   } else if (value instanceof Long) {
+   value = Int64Value.of((long) value).toByteString();
+   } else if (value instanceof Float) {
+   value = FloatValue.of((float) value).toByteString();
+   } else if (value instanceof Double) {
+   value = DoubleValue.of((double) value).toByteString();
+   }

Review Comment:
   So I tried to consider the case where we send an Avro ``int`` into a 
``NUMERIC`` column for BigQuery but this is completely messing things up. I can 
add something like:
   java
} else if (value instanceof Integer) {
   value = Int64Value.of((int) value).toByteString();
}
   
   But then, in the test when checking the returned value:
   java
   assertEquals(firstElt.get(0).getDoubleValue(), 1.0);
   
   It would tell me:
   
   org.opentest4j.AssertionFailedError: expected: <2.64E-7> but was: <1.0>
   
   
   If sending the Avro ``int`` into a ``INT64`` column for BigQuery, then it 
works as expected.
   
   I'm going to push a commit containing the additional else if so that it does 
not completely fail in NiFi but I believe this would push wrong data into the 
BQ table.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.


[GitHub] [nifi] pvillard31 commented on a diff in pull request #7489: NIFI-11823 - fix NUMERIC support in PutBigQuery

2023-08-02 Thread via GitHub


pvillard31 commented on code in PR #7489:
URL: https://github.com/apache/nifi/pull/7489#discussion_r1282393730


##
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/bigquery/proto/ProtoUtils.java:
##
@@ -17,50 +17,99 @@
 
 package org.apache.nifi.processors.gcp.bigquery.proto;
 
+import com.google.cloud.bigquery.storage.v1.BigDecimalByteStringEncoder;
+import com.google.cloud.bigquery.storage.v1.TableFieldSchema.Type;
+import com.google.cloud.bigquery.storage.v1.TableSchema;
 import com.google.protobuf.Descriptors;
+import com.google.protobuf.DoubleValue;
 import com.google.protobuf.DynamicMessage;
+import com.google.protobuf.FloatValue;
+import com.google.protobuf.Int64Value;
+
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Map;
 
 /**
- * Util class for protocol buffer messaging
- */
+* Util class for protocol buffer messaging
+*/
 public class ProtoUtils {
 
-public static DynamicMessage createMessage(Descriptors.Descriptor 
descriptor, Map valueMap) {
-DynamicMessage.Builder builder = DynamicMessage.newBuilder(descriptor);
+   public static DynamicMessage createMessage(Descriptors.Descriptor 
descriptor, Map valueMap, TableSchema tableSchema) {
+   final DynamicMessage.Builder builder = 
DynamicMessage.newBuilder(descriptor);
+
+   for (final Descriptors.FieldDescriptor field : descriptor.getFields()) {
+   final String name = field.getName();
+   Object value = valueMap.get(name);
+   if (value == null) {
+   continue;
+   }
+
+   switch (field.getType()) {
+   case MESSAGE:
+   if (field.isRepeated()) {
+   Collection collection = value.getClass().isArray() ? 
Arrays.asList((Object[]) value) : (Collection) value;
+   collection.forEach(act -> builder.addRepeatedField(field, 
createMessage(field.getMessageType(), (Map) act, tableSchema)));
+   } else {
+   builder.setField(field, 
createMessage(field.getMessageType(), (Map) value, 
tableSchema));
+   }
+   break;
+
+   // INT64 with alias INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT
+   case INT64:
+   // Integer in the bigquery table schema maps back to INT64 
which is considered to be Long on Java side:
+   // https://developers.google.com/protocol-buffers/docs/proto3
+   if (value instanceof Integer) {
+   value = Long.valueOf((Integer) value);
+   }
+
+   setField(value, field, builder);
+   break;
+
+   // FLOAT64
+   case DOUBLE:
+   if (value instanceof Float) {
+   value = ((Float) value).doubleValue();
+   }
+   setField(value, field, builder);
+   break;
+
+   // matches NUMERIC and BIGNUMERIC types in BigQuery
+   // BQTableSchemaToProtoDescriptor.class
+   case BYTES:
+   if(value instanceof BigDecimal) {
+   
if(tableSchema.getFields(field.getIndex()).getType().equals(Type.BIGNUMERIC)) {

Review Comment:
   Fixed, thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@nifi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org