You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/09/21 11:44:23 UTC

[GitHub] [iceberg] massdosage commented on a change in pull request #1407: Hive: HiveIcebergOutputFormat first implementation for handling Hive inserts into unpartitioned Iceberg tables

massdosage commented on a change in pull request #1407:
URL: https://github.com/apache/iceberg/pull/1407#discussion_r491924634



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergWritable.java
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.mapreduce;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import org.apache.hadoop.io.Writable;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class IcebergWritable implements Writable {
+  private Record record;
+
+  public IcebergWritable(Record record) {
+    this.record = record;
+  }
+
+  public Record record() {
+    Preconditions.checkNotNull(record, "Should not return null record");
+    return record;
+  }
+
+  public Object getValueObject(int colIndex) {
+    if (record != null) {
+      return record.get(colIndex);
+    } else {
+      return null;
+    }
+  }
+
+  public Object getValueObject(String colName) {
+    if (record != null) {
+      return record.getField(colName);
+    } else {
+      return null;
+    }
+  }
+
+  public boolean isSet(int colIndex) {
+    if (record != null) {
+      return record.get(colIndex) != null;
+    } else {
+      return false;
+    }
+  }
+
+  public boolean isSet(String colName) {
+    if (record != null) {
+      return record.getField(colName) != null;
+    } else {
+      return false;
+    }
+  }
+
+//    public void populateRow(PartialRow row) {

Review comment:
       If this isn't needed it's probably best to remove the commented out code.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveRunnerWrite.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.hive;
+
+import com.klarna.hiverunner.HiveShell;
+import com.klarna.hiverunner.StandaloneHiveRunner;
+import com.klarna.hiverunner.annotations.HiveSQL;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.TestHelper;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(StandaloneHiveRunner.class)
+public class TestHiveRunnerWrite {

Review comment:
       From what I can see this covers all the existing variations on inserting data into a table that already exists. I've confirmed running variations of all these in a "real", distributed Hive setup and they seem to work.

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveRunnerWrite.java
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.hive;
+
+import com.klarna.hiverunner.HiveShell;
+import com.klarna.hiverunner.StandaloneHiveRunner;
+import com.klarna.hiverunner.annotations.HiveSQL;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.TestHelper;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(StandaloneHiveRunner.class)
+public class TestHiveRunnerWrite {
+  private static final Schema SCHEMA = new Schema(
+      required(1, "data", Types.StringType.get()),
+      required(2, "id", Types.LongType.get()));
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @HiveSQL(files = {})
+  private HiveShell shell;
+
+  private static final Configuration conf = new Configuration();
+  private TestHelper helper;
+  private Table table;
+
+  @Before
+  public void before() throws Exception {
+    helper = new TestHelper(conf,
+        new HadoopTables(conf),
+        temp.newFolder(FileFormat.PARQUET.name()).toString(),
+        SCHEMA,
+        null,
+        FileFormat.PARQUET,
+        temp);
+
+    table = helper.createUnpartitionedTable();
+
+    shell.executeQuery("CREATE TABLE withShell STORED BY '" + HiveIcebergStorageHandler.class.getName() + "' " +
+        "LOCATION '" + table.location() + "' " +
+        "TBLPROPERTIES ('" + InputFormatConfig.WRITE_FILE_FORMAT + "'='" + FileFormat.PARQUET.name() + "')");
+  }
+
+  @After
+  public void after() {
+    shell.executeQuery("DROP TABLE withShell");
+  }
+
+  @Test
+  public void testInsert() {
+    List<Record> records = helper.generateRandomRecords(2, 0L);
+    // The expected query is like
+    // INSERT INTO withShell VALUES ('farkas', 1), ('kutya', 2)
+    StringBuilder query = new StringBuilder().append("INSERT INTO withShell VALUES ");
+    records.forEach(record -> query.append("('").append(record.get(0)).append("',").append(record.get(1)).append("),"));
+    query.setLength(query.length() - 1);
+
+    shell.executeQuery(query.toString());
+
+    HiveIcebergSerDeTestUtils.validate(table, records, 1);
+  }
+
+  @Test
+  public void testInsertFromSelect() throws IOException {
+    List<Record> records = helper.generateRandomRecords(2, 0L);
+    helper.appendToTable(null, records);
+
+    // Just check the initial data
+    HiveIcebergSerDeTestUtils.validate(table, records, 1);
+
+    shell.executeQuery("INSERT INTO withShell SELECT * FROM withShell");
+
+    // Check that everything is duplicated as expected
+    records.addAll(records);
+    HiveIcebergSerDeTestUtils.validate(table, records, 1);
+  }
+
+  @Test
+  public void testInsertFromSelectWithOrderBy() throws IOException {
+    // We expect that there will be Mappers and Reducers here
+    List<Record> records = helper.generateRandomRecords(2, 0L);
+    helper.appendToTable(null, records);
+
+    // Just check the initial data
+    HiveIcebergSerDeTestUtils.validate(table, records, 1);
+
+    shell.executeQuery("INSERT INTO withShell SELECT * FROM withShell ORDER BY id");
+
+    // Check that everything is duplicated as expected
+    records.addAll(records);
+    HiveIcebergSerDeTestUtils.validate(table, records, 1);
+  }
+
+  @Test
+  public void testDefaultFileFormat() {
+    shell.executeQuery("CREATE TABLE withShell2 STORED BY '" + HiveIcebergStorageHandler.class.getName() + "' " +
+        "LOCATION '" + table.location() + "'");
+
+    List<Record> records = helper.generateRandomRecords(2, 0L);
+    // The expected query is like
+    // INSERT INTO withShell VALUES ('farkas', 1), ('kutya', 2)
+    StringBuilder query = new StringBuilder().append("INSERT INTO withShell2 VALUES ");
+    records.forEach(record -> query.append("('").append(record.get(0)).append("',").append(record.get(1)).append("),"));
+    query.setLength(query.length() - 1);
+
+    shell.executeQuery(query.toString());

Review comment:
       Don't you want to query this back to check they were inserted correctly?

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputFormat.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.hive;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.JobContextImpl;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptContextImpl;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.TestHelper;
+import org.apache.iceberg.mr.hive.HiveIcebergOutputFormat.IcebergRecordWriter;
+import org.apache.iceberg.mr.mapreduce.IcebergWritable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestHiveIcebergOutputFormat {
+
+  private static final Object[] TESTED_FILE_FORMATS = new Object[] {"avro", "orc", "parquet"};
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Object[] parameters() {
+    return TESTED_FILE_FORMATS;
+  }
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  private static final Configuration conf = new Configuration();
+  private TestHelper helper;
+  private Table table;
+  private TestOutputFormat testOutputFormat;
+
+  public TestHiveIcebergOutputFormat(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws Exception {
+    helper = new TestHelper(conf,
+        new HadoopTables(conf),
+        temp.newFolder(this.fileFormat.name()).toString(),
+        HiveIcebergSerDeTestUtils.FULL_SCHEMA,
+        null,
+        fileFormat,
+        temp);
+
+    table = helper.createUnpartitionedTable();
+    testOutputFormat = new TestOutputFormat(table, fileFormat);
+  }
+
+  @Test
+  public void testWriteRow() throws IOException {
+    // Write a row.
+    List<Record> records =
+        Arrays.asList(new Record[] { HiveIcebergSerDeTestUtils.getTestRecord(FileFormat.PARQUET.equals(fileFormat)) });
+
+    testOutputFormat.write(records, false, false);
+    testOutputFormat.validate(records);
+  }
+
+  @Test
+  public void testNullRow() throws IOException {
+    // FIXME: ORC file does not read back the row consisting only of nulls. The data in the files seems ok.
+    if (FileFormat.ORC.equals(fileFormat)) {
+      return;
+    }
+    // Write a row.
+    List<Record> records = Arrays.asList(new Record[] { HiveIcebergSerDeTestUtils.getNullTestRecord() });
+
+    testOutputFormat.write(records, false, false);
+    testOutputFormat.validate(records);
+  }
+
+  @Test
+  public void testMultipleRows() throws IOException {
+    // Write 2 rows. One with nulls too.
+    List<Record> records = Arrays.asList(new Record[] {
+        HiveIcebergSerDeTestUtils.getTestRecord(FileFormat.PARQUET.equals(fileFormat)),
+        HiveIcebergSerDeTestUtils.getNullTestRecord()
+    });
+
+    testOutputFormat.write(records, false, false);
+    testOutputFormat.validate(records);
+  }
+
+  @Test
+  public void testRandomRecords() throws IOException {
+    // Write 30 random rows
+    // FIXME: Parquet appender expect byte[] instead of UUID when writing values.

Review comment:
       Is this something you still want to fix in this PR or in another one?

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputFormat.java
##########
@@ -0,0 +1,285 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr.hive;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.JobContextImpl;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptContextImpl;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.TestHelper;
+import org.apache.iceberg.mr.hive.HiveIcebergOutputFormat.IcebergRecordWriter;
+import org.apache.iceberg.mr.mapreduce.IcebergWritable;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestHiveIcebergOutputFormat {
+
+  private static final Object[] TESTED_FILE_FORMATS = new Object[] {"avro", "orc", "parquet"};
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Object[] parameters() {
+    return TESTED_FILE_FORMATS;
+  }
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  private static final Configuration conf = new Configuration();
+  private TestHelper helper;
+  private Table table;
+  private TestOutputFormat testOutputFormat;
+
+  public TestHiveIcebergOutputFormat(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws Exception {
+    helper = new TestHelper(conf,
+        new HadoopTables(conf),
+        temp.newFolder(this.fileFormat.name()).toString(),
+        HiveIcebergSerDeTestUtils.FULL_SCHEMA,
+        null,
+        fileFormat,
+        temp);
+
+    table = helper.createUnpartitionedTable();
+    testOutputFormat = new TestOutputFormat(table, fileFormat);
+  }
+
+  @Test
+  public void testWriteRow() throws IOException {
+    // Write a row.
+    List<Record> records =
+        Arrays.asList(new Record[] { HiveIcebergSerDeTestUtils.getTestRecord(FileFormat.PARQUET.equals(fileFormat)) });
+
+    testOutputFormat.write(records, false, false);
+    testOutputFormat.validate(records);
+  }
+
+  @Test
+  public void testNullRow() throws IOException {
+    // FIXME: ORC file does not read back the row consisting only of nulls. The data in the files seems ok.

Review comment:
       Is this something you still want to fix in this PR or in another one?




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org