-
Notifications
You must be signed in to change notification settings - Fork 3k
Implement Hive input format #1192
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
|
||
| // The table location of the split allows Hive to map a split to a table and/or partition. | ||
| // See calls to `getPartitionDescFromPathRecursively` in `CombineHiveInputFormat` or `HiveInputFormat`. | ||
| private String tableLocation; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like what's happening is the table location is used as the split's path so that Hive associates all splits with the same PartitionDesc that contains a TableDesc. Is that correct? If so, I think it would be better to add that as the comment. It's difficult to read the Hive code and figure out what's going on using just the pointers here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hive uses the path name of the split to map it back to a PartitionDesc or TableDesc, which specify the relevant input format for reading the files belonging to that partition or table. That way, HiveInputFormat and CombineHiveInputFormat can read files with different input formats in the same MR job and combine compatible splits together.
I'll update the comment.
| public void write(DataOutput dataOutput) { | ||
| throw new UnsupportedOperationException("write is not supported."); | ||
| public void readFields(DataInput in) throws IOException { | ||
| throw new UnsupportedOperationException("readFields is not supported"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think these last two functions need to change?
| /** | ||
| * Generic Mrv1 InputFormat API for Iceberg. | ||
| * | ||
| * @param <T> T is the in memory data model which can either be Pig tuples, Hive rows. Default is Iceberg records |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could you update this to Java class of records constructed by Iceberg; default is {@link Record}?
It is odd that this currently states that T could be either A or B, but defaults to C.
|
|
||
| static TaskAttemptContext getTaskAttemptContext(JobConf job) { | ||
| TaskAttemptID taskAttemptID = Optional.ofNullable(TaskAttemptID.forName(job.get("mapred.task.id"))) | ||
| .orElse(new TaskAttemptID()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When would mapred.task.id be null? Should we throw an exception in that case?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm guessing that this is because an attempt context is passed as a JobContext. Let's fix that problem and then we won't need to do this. The helpers I mentioned also demonstrate how to create a TaskAttemptContext from specific values.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I looked into this and Hive (when spawning a local MR job) or YARN populates the mapreduce.task.id and mapreduce.task.attempt.id" (among many other properties). So I changed this line to: TaskAttemptID.forName(job.get("mapreduce.task.attempt.id"). Several input formats in the Hive codebase do the same.
I believe it is not the responsibility of the input format to create a TaskAttemptID from scratch and setting the mapred{uce}* properties. The framework using the input format are responsible for that.
During my tests, I've seen that the task attempt it is not set only when Hive uses the the file input format outside of a MR job (single fetch task). This is when we need to fallback to the default constructor.
The same logic applies for the JobContext object.
|
|
||
| @Override | ||
| public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { | ||
| return innerInputFormat.getSplits(getTaskAttemptContext(job)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
getSplits accepts a JobContext and I think it makes sense to pass objects that are as close as possible to what the mapreduce framework would use. We have some helper methods in our branch for reading Hive tables from Spark's DSv2 that you might want to check out: https://github.com/Netflix/iceberg/blob/netflix-spark-2.4/metacat/src/main/java/com/netflix/iceberg/batch/MapReduceUtil.java.
Those can help you create mapreduce objects after inspecting the mapred objects.
|
Looks good to me. I made a few comments, but I don't think there is anything major to fix. It would be nice to fix some of the comments and to instantiate the class, |
|
@massdosage & @guilload, is the plan to add the |
I'm taking a look at this now, but yes, I should be able to add the |
|
@guilload Thanks for raising the InputFormat PR, I've been taking a look at it today to try get some HiveRunner tests working against it. I checked out the branch from the PR and did the minimal I could to get a test running that inserts some data and then returns it, essentially this: with some changes to the class of the input format (and ignoring all the other test in that class). Unfortunately this is failing for me with exceptions like so: Line 186 in the mapreduce input format is doing this:
How did you manage to get the HiveRunner tests passing? Am I missing something? |
e55aa74 to
f3928c7
Compare
|
@rdblue Thanks for the review. I'll make the changes you suggested tomorrow. @massdosage, please take a look at my latest commit. |
9351f18 to
23ad535
Compare
| schema = table.schema(); | ||
|
|
||
| List<String> projectedColumns = parseProjectedColumns(job); | ||
| projection = projectedColumns.isEmpty() ? schema : projection.select(projectedColumns); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
| projection = projectedColumns.isEmpty() ? schema : projection.select(projectedColumns); | |
| projection = projectedColumns.isEmpty() ? schema : schema.select(projectedColumns); |
Shouldn't the code be like above? If not I get a NPE like this when running a HiveRunner test:
Caused by:
java.lang.NullPointerException
at org.apache.iceberg.mr.hive.HiveIcebergInputFormat.getSplits(HiveIcebergInputFormat.java:58)
at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextSplits(FetchOperator.java:372)
at org.apache.hadoop.hive.ql.exec.FetchOperator.getRecordReader(FetchOperator.java:304)
at org.apache.hadoop.hive.ql.exec.FetchOperator.getNextRow(FetchOperator.java:459)
... 13 more
I merged in your changes from yesterday, I've added a comment above on a NPE I ran into. I then also get the following error when I run the equivalent of this test: |
23ad535 to
b373262
Compare
mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergInputFormat.java
Outdated
Show resolved
Hide resolved
mr/src/main/java/org/apache/iceberg/mr/mapred/MapredIcebergInputFormat.java
Outdated
Show resolved
Hide resolved
mr/src/main/java/org/apache/iceberg/mr/mapred/MapredIcebergInputFormat.java
Show resolved
Hide resolved
3cdd9f1 to
61859cc
Compare
|
|
||
| @Override | ||
| public List<InputSplit> getSplits(JobContext context) { | ||
| if (splits != null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hive keeps a cache of input format instance arounds (see HiveInputFormat) that breaks this logic so I've chosen to remove it for now.
We can re-implement this later but the logic will have to be a bit more robust.
61859cc to
f535af0
Compare
|
@massdosage Sorry, I broke this branch multiple times in the past few days but this PR is stable and in good shape now. I've added a short HiveRunner test to ensure that this won't happen again. You can extend it or remove it later when you add your own tests. In addition, you can check out this two commits 10b62fc and f2232b8. In the former, I branched off of your @rdblue @rdsr I've made the changes you suggested and I believe this PR is ready for a second round of review. |
No problem, I took a quick look and the tests you've added look good, I think it makes sense to have them directly in this PR to show that it all works. I don't have time today but I'll try take a proper look on Monday. I'd also like to try out your branch on a Hive client to check everything works end to end on a full Hadoop cluster, not sure if you've also done that yet? |
|
The tests look good and work for me. I'm now trying to get them running from a Hive client. I've built all the Iceberg jars and uploaded them to the host with the Hive client. I've then created a table using Iceberg from Spark. I've now opened a Hive session and done the following to add the jars to the classpath: I get what look like successful responses from Hive at this point: I then try create a table something like so: However I then see this error in the Hive user logs: |
|
@massdosage, yes I have managed to use the input format in a "real" Hive client. I'm taking a shortcut though, I just add |
|
@guilload OK, thanks, let me know what you find. It has made me think we should probably have this mr module produce an "uber jar" so for the Hive use case you just have to add 1 jar to the classpath, not 6. |
| // Adding the ORDER BY clause will cause Hive to spawn a local MR job this time. | ||
| List<Object[]> descRows = shell.executeStatement("SELECT * FROM default.customers ORDER BY customer_id DESC"); | ||
|
|
||
| Assert.assertEquals(3, rows.size()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be Assert.assertEquals(3, descRows.size());
|
The changes LGTM! |
|
Thanks for reviewing, @rdsr! |
This PR implements:
A few things:
IcebergSplitclass out offIcebergInputFormatand the class now extendsmapreduce.InputSplitand implementsmapred.InputSplitso it can be returned by both MR v1 and v2 input formats.IcebergSplitContainerinterface trick that I used to avoid overridinggetRecordReaderin subclasses ofMapredIcebergInputFormat. Recommendations for a more elegant way to do so are welcome.TestIcebergInputFormatclass quite a bit so it be can run against the two MR file formats implementations.@rdblue @rdsr @massdosage @cmathiesen
cc @edgarRd @gustavoatt