Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,9 @@ public static Object toFlinkObject(ObjectInspector inspector, Object data, HiveS
if (inspector instanceof ListObjectInspector) {
ListObjectInspector listInspector = (ListObjectInspector) inspector;
List<?> list = listInspector.getList(data);
if (list == null) {
return null;
}

// flink expects a specific array type (e.g. Integer[] instead of Object[]), so we have
// to get the element class
Expand All @@ -332,6 +335,9 @@ public static Object toFlinkObject(ObjectInspector inspector, Object data, HiveS
if (inspector instanceof MapObjectInspector) {
MapObjectInspector mapInspector = (MapObjectInspector) inspector;
Map<?, ?> map = mapInspector.getMap(data);
if (map == null) {
return null;
}

Map<Object, Object> result = new HashMap<>(map.size());
for (Map.Entry<?, ?> entry : map.entrySet()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -531,6 +531,31 @@ public void testLocationWithComma() throws Exception {
}
}

@Test
public void testReadHiveDataWithEmptyMapForHiveShim20X() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can only run this test for hive <=2.0.0. Something like Assume.assumeTrue(HiveShimLoader.getHiveVersion().compareTo("2.0.0") <= 0)

Assume.assumeTrue(HiveShimLoader.getHiveVersion().compareTo("2.0.0") <= 0);
TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
try {
String format = "parquet";
// test.parquet data: hehuiyuan {} []
String folderURI = this.getClass().getResource("/parquet").getPath();

tableEnv.getConfig()
.getConfiguration()
.set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
tableEnv.executeSql(
String.format(
"create external table src_t (a string, b map<string, string>, c array<string>) stored as %s location 'file://%s'",
format, folderURI));

List<Row> results =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's also verify the results

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we should also test empty arrays?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes ,the same problem

CollectionUtil.iteratorToList(
tableEnv.sqlQuery("select * from src_t").execute().collect());
} finally {
tableEnv.executeSql("drop table if exists src_t");
}
}

private TableEnvironment getTableEnvWithHiveCatalog() {
TableEnvironment tableEnv =
HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(SqlDialect.HIVE);
Expand Down
Binary file not shown.