You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/09/03 16:36:08 UTC

[GitHub] [nifi] exceptionfactory commented on a change in pull request #5358: NIFI-9185 dd Avro logical type to SelectHive3QL processor

exceptionfactory commented on a change in pull request #5358:
URL: https://github.com/apache/nifi/pull/5358#discussion_r702030424



##########
File path: nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestSelectHive3QL.java
##########
@@ -690,6 +698,114 @@ public void testMaxRowsPerFlowFileWithMaxFragments() throws ClassNotFoundExcepti
         runner.clearTransferState();
     }
 
+    @Test
+    public void testAvroRecordCreatedWithoutLogicalTypesByDefault() throws SQLException, IOException {
+        final Schema expectedSchema = SchemaBuilder.record("NiFi_SelectHiveQL_Record").namespace("any.data").fields()
+                .name("ID").type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault()
+                .name("NAME").type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault()
+                .name("BIRTH_DATE").type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault()
+                .name("BIG_NUMBER").type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault()
+                .name("CREATED_ON").type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault()
+                .endRecord();
+        final int expectedId = 1;
+        final String expectedName = "Joe Smith";
+        final String expectedBirthDate = "1956-11-22";
+        final String expectedBigNumber = "12345678.12";
+        final String expectedCreatedOn = "1962-09-23 03:23:34.234";
+
+        // load test data to database
+        final Connection con = ((DBCPService) runner.getControllerService("dbcp")).getConnection();
+        final Statement stmt = con.createStatement();
+        final InputStream in;
+        final MockFlowFile mff;
+
+        try {
+            stmt.execute("drop table TEST_QUERY_DB_TABLE");
+        } catch (final SQLException sqle) {
+            // Ignore this error, probably a "table does not exist" since Derby doesn't yet support DROP IF EXISTS [DERBY-4842]
+        }
+
+        stmt.execute("create table TEST_QUERY_DB_TABLE (id integer not null, name varchar(100), birth_date date, big_number decimal(10,2),created_on timestamp)");
+        stmt.execute("insert into TEST_QUERY_DB_TABLE (id, name, birth_date, big_number, created_on) VALUES (1, 'Joe Smith', '1956-11-22', 12345678.12, '1962-09-23 03:23:34.234')");
+
+        runner.setIncomingConnection(false);
+        runner.setProperty(SelectHive3QL.HIVEQL_SELECT_QUERY, "SELECT * FROM TEST_QUERY_DB_TABLE");
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(SelectHive3QL.REL_SUCCESS, 1);
+        mff = runner.getFlowFilesForRelationship(SelectHive3QL.REL_SUCCESS).get(0);
+        in = new ByteArrayInputStream(mff.toByteArray());
+
+        final GenericRecord record = getFirstRecordFromStream(in);
+
+        assertEquals(expectedSchema, record.getSchema());
+        assertEquals(expectedId, record.get("ID"));
+        assertEquals(expectedName, record.get("NAME").toString());
+        assertEquals(expectedBirthDate, record.get("BIRTH_DATE").toString());
+        assertEquals(expectedBigNumber, record.get("BIG_NUMBER").toString());
+        assertEquals(expectedCreatedOn, record.get("CREATED_ON").toString());
+
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testAvroRecordCreatedWithLogicalTypesWhenSet() throws SQLException, IOException {
+        final Schema expectedSchema = SchemaBuilder.record("NiFi_SelectHiveQL_Record").namespace("any.data").fields()
+                .name("ID").type().unionOf().nullBuilder().endNull().and().intType().endUnion().noDefault()
+                .name("NAME").type().unionOf().nullBuilder().endNull().and().stringType().endUnion().noDefault()
+                .name("BIRTH_DATE").type().unionOf().nullBuilder().endNull().and()
+                .type(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT))).endUnion().noDefault()
+                .name("BIG_NUMBER").type().unionOf().nullBuilder().endNull().and()
+                .type(LogicalTypes.decimal(10, 2).addToSchema(Schema.create(Schema.Type.BYTES))).endUnion().noDefault()
+                .name("CREATED_ON").type().unionOf().nullBuilder().endNull().and()
+                .type(LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG))).endUnion().noDefault()
+                .endRecord();
+        final int expectedId = 1;
+        final String expectedName = "Joe Smith";
+        final int expectedBirthDate = (int) LocalDate.parse("1956-11-22").toEpochDay();
+        final BigDecimal decimal = new BigDecimal("12345678.12").setScale(2, BigDecimal.ROUND_HALF_EVEN);
+        final ByteBuffer expectedBigNumber = ByteBuffer.wrap(decimal.unscaledValue().toByteArray());
+        final long expectedCreatedOn = LocalDateTime.parse("1962-09-23T03:23:34.234").atZone(ZoneId.systemDefault()).withZoneSameInstant(ZoneId.of("UTC")).toInstant().toEpochMilli();

Review comment:
       This test value comparison looks like a potential problem given recent updates to timestamp conversion for NIFI-8749 in #5361. The number of methods also makes it very difficult to follow, so at minimum, it would be helpful to break out the conversion into separate variables.
   
   Converting from LocalDateTime using `ZoneId.systemDefault()` will behave different depending on the system time zone, time zone conversion in test methods should be avoided if at all possible. It would be much better to declare the expected timestamp with a time zone set to UTC and then use that for the input and output epoch milliseconds.




-- 
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-unsubscribe@nifi.apache.org

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