pvary commented on pull request #1407:
URL: https://github.com/apache/iceberg/pull/1407#issuecomment-686440819
Here comes the new implementation with the mapred.OutputCommitter.
If the previous one was The Bad, then this one is The Ugly. 😃
What I have learned:
- mapred.OutputCommitter has to be a static class - and it has to have a
default constructor
```
2020-09-03T13:03:10,637 INFO [Thread-100] mapred.LocalJobRunner: Failed to
createOutputCommitter
java.lang.RuntimeException: java.lang.NoSuchMethodException:
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat$IcebergOutputCommitter.<init>()
at
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:135)
~[hadoop-common-3.1.0.jar:?]
at
org.apache.hadoop.mapred.LocalJobRunner$Job.createOutputCommitter(LocalJobRunner.java:515)
~[hadoop-mapreduce-client-common-3.1.0.jar:?]
at
org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:530)
~[hadoop-mapreduce-client-common-3.1.0.jar:?]
Caused by: java.lang.NoSuchMethodException:
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat$IcebergOutputCommitter.<init>()
at java.lang.Class.getConstructor0(Class.java:3082) ~[?:1.8.0_152]
at java.lang.Class.getDeclaredConstructor(Class.java:2178)
~[?:1.8.0_152]
at
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:129)
~[hadoop-common-3.1.0.jar:?]
```
- There is a method helping to configure jobs which are very handy if
specific configuration is needed. See:
HiveIcebergStorageHandler.configureJobConf. We can use this to set the
OutputCommitter - otherwise Hive uses a default NullOutputCommitter.
- There is a different instance of HiveIcebergOutputFormat on HiveServer2,
and on the Mappers/Reducers. Reflection is used to create it on the
Mapper/Reducer side from scratch, so the only way to send info to it is
JobConf. See: HiveIcebergStorageHandler.configureOutputJobProperties
- getHiveRecordWriter will be called on Mapper/Reducer side, so we do not
want to load table there 😄
- We might be able to communicate between RecordWriter and OutputCommitter
on Mapper/Reducer side through JobConf/TaskAttemptContext.jobConf, but I am not
too comfortable with it - Any thoughts here?
The basic idea behind the patch is:
1. We need to use HiveIcebergStorageHandler.configureJobConf to set the
mapred.output.committer.class.
2. Writers are creating the files for a directory defined by the table and
the hive query id. When a writer is closed it closes the appender and creates a
committed file for marking the file finished.
3. When the job is finished it lists the directory and appends the data
files to the table.
What I do not like:
- Relies on listing of the target dir. In Hive we do this often, and on S3
we have to use S3Guard to have a reliable listing. I was hoping to get rid of
this at least for Iceberg tables. @rdblue, @massdosage: Do you think it would
be an acceptable compromise for Hive writes? Do you have any better ideas?
- Creating the commit file would be better placed in commitTask, but
serializing the Metrics into the JobConf object seems like a bad idea. @rdblue,
@massdosage: Any thoughts?
What I have found:
- org.apache.iceberg.Metrics implements Serializable but uses ByteBuffers
which are not Serializable, so I had to create a serialized version for it.
Thanks,
Peter
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]