[HUDI-1080] Fix backward compatibility for com.uber inputformats
This commit is contained in:
@@ -0,0 +1,60 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.uber.hoodie.hadoop.hive;
|
||||||
|
|
||||||
|
import com.uber.hoodie.hadoop.HoodieInputFormat;
|
||||||
|
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||||
|
import org.apache.hadoop.io.Writable;
|
||||||
|
import org.apache.hadoop.io.WritableComparable;
|
||||||
|
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||||
|
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||||
|
|
||||||
|
public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extends Writable>
|
||||||
|
extends org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat<K, V> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getParquetInputFormatClassName() {
|
||||||
|
return HoodieInputFormat.class.getName();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getParquetRealtimeInputFormatClassName() {
|
||||||
|
return HoodieRealtimeInputFormat.class.getName();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim
|
||||||
|
createInputFormatShim() {
|
||||||
|
return new HoodieCombineFileInputFormatShim<>();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class HoodieCombineFileInputFormatShim<K, V>
|
||||||
|
extends org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected HoodieParquetInputFormat createParquetInputFormat() {
|
||||||
|
return new HoodieInputFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected HoodieParquetRealtimeInputFormat createParquetRealtimeInputFormat() {
|
||||||
|
return new HoodieRealtimeInputFormat();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -20,7 +20,6 @@ package org.apache.hudi.hadoop.hive;
|
|||||||
|
|
||||||
import org.apache.hudi.common.util.ReflectionUtils;
|
import org.apache.hudi.common.util.ReflectionUtils;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
|
||||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||||
import org.apache.hudi.hadoop.realtime.HoodieCombineRealtimeRecordReader;
|
import org.apache.hudi.hadoop.realtime.HoodieCombineRealtimeRecordReader;
|
||||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
||||||
@@ -109,6 +108,18 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
private static final int MAX_CHECK_NONCOMBINABLE_THREAD_NUM = 50;
|
private static final int MAX_CHECK_NONCOMBINABLE_THREAD_NUM = 50;
|
||||||
private static final int DEFAULT_NUM_PATH_PER_THREAD = 100;
|
private static final int DEFAULT_NUM_PATH_PER_THREAD = 100;
|
||||||
|
|
||||||
|
protected String getParquetInputFormatClassName() {
|
||||||
|
return HoodieParquetInputFormat.class.getName();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String getParquetRealtimeInputFormatClassName() {
|
||||||
|
return HoodieParquetRealtimeInputFormat.class.getName();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HoodieCombineFileInputFormatShim createInputFormatShim() {
|
||||||
|
return new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim<>();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create Hive splits based on CombineFileSplit.
|
* Create Hive splits based on CombineFileSplit.
|
||||||
*/
|
*/
|
||||||
@@ -118,8 +129,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
|
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
|
||||||
Map<String, Operator<? extends OperatorDesc>> aliasToWork = mrwork.getAliasToWork();
|
Map<String, Operator<? extends OperatorDesc>> aliasToWork = mrwork.getAliasToWork();
|
||||||
/* MOD - Initialize a custom combine input format shim that will call listStatus on the custom inputFormat **/
|
/* MOD - Initialize a custom combine input format shim that will call listStatus on the custom inputFormat **/
|
||||||
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim combine =
|
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim combine = createInputFormatShim();
|
||||||
new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim<>();
|
|
||||||
|
|
||||||
InputSplit[] splits;
|
InputSplit[] splits;
|
||||||
|
|
||||||
@@ -151,9 +161,9 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
|
InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
|
||||||
LOG.info("Input Format => " + inputFormatClass.getName());
|
LOG.info("Input Format => " + inputFormatClass.getName());
|
||||||
// **MOD** Set the hoodie filter in the combine
|
// **MOD** Set the hoodie filter in the combine
|
||||||
if (inputFormatClass.getName().equals(HoodieParquetInputFormat.class.getName())) {
|
if (inputFormatClass.getName().equals(getParquetInputFormatClassName())) {
|
||||||
combine.setHoodieFilter(true);
|
combine.setHoodieFilter(true);
|
||||||
} else if (inputFormatClass.getName().equals(HoodieParquetRealtimeInputFormat.class.getName())) {
|
} else if (inputFormatClass.getName().equals(getParquetRealtimeInputFormatClassName())) {
|
||||||
LOG.info("Setting hoodie filter and realtime input format");
|
LOG.info("Setting hoodie filter and realtime input format");
|
||||||
combine.setHoodieFilter(true);
|
combine.setHoodieFilter(true);
|
||||||
combine.setRealTime(true);
|
combine.setRealTime(true);
|
||||||
@@ -540,16 +550,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
|
|
||||||
pushProjectionsAndFilters(job, inputFormatClass, hsplit.getPath(0));
|
pushProjectionsAndFilters(job, inputFormatClass, hsplit.getPath(0));
|
||||||
|
|
||||||
if (inputFormatClass.getName().equals(HoodieParquetInputFormat.class.getName())) {
|
if (inputFormatClass.getName().equals(getParquetRealtimeInputFormatClassName())) {
|
||||||
return ShimLoader.getHadoopShims().getCombineFileInputFormat().getRecordReader(job, (CombineFileSplit) split,
|
HoodieCombineFileInputFormatShim shims = createInputFormatShim();
|
||||||
reporter, CombineHiveRecordReader.class);
|
|
||||||
} else if (inputFormatClass.getName().equals(HoodieParquetRealtimeInputFormat.class.getName())) {
|
|
||||||
HoodieCombineFileInputFormatShim shims = new HoodieCombineFileInputFormatShim();
|
|
||||||
IOContextMap.get(job).setInputPath(((CombineHiveInputSplit) split).getPath(0));
|
IOContextMap.get(job).setInputPath(((CombineHiveInputSplit) split).getPath(0));
|
||||||
return shims.getRecordReader(job, ((CombineHiveInputSplit) split).getInputSplitShim(),
|
return shims.getRecordReader(job, ((CombineHiveInputSplit) split).getInputSplitShim(),
|
||||||
reporter, CombineHiveRecordReader.class);
|
reporter, CombineHiveRecordReader.class);
|
||||||
} else {
|
} else {
|
||||||
throw new HoodieException("Unexpected input format : " + inputFormatClassName);
|
return ShimLoader.getHadoopShims().getCombineFileInputFormat().getRecordReader(job, (CombineFileSplit) split,
|
||||||
|
reporter, CombineHiveRecordReader.class);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -834,6 +842,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
private boolean hoodieFilter = false;
|
private boolean hoodieFilter = false;
|
||||||
private boolean isRealTime = false;
|
private boolean isRealTime = false;
|
||||||
|
|
||||||
|
protected HoodieParquetInputFormat createParquetInputFormat() {
|
||||||
|
return new HoodieParquetInputFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected HoodieParquetRealtimeInputFormat createParquetRealtimeInputFormat() {
|
||||||
|
return new HoodieParquetRealtimeInputFormat();
|
||||||
|
}
|
||||||
|
|
||||||
public HoodieCombineFileInputFormatShim() {
|
public HoodieCombineFileInputFormatShim() {
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -864,10 +880,10 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
HoodieParquetInputFormat input;
|
HoodieParquetInputFormat input;
|
||||||
if (isRealTime) {
|
if (isRealTime) {
|
||||||
LOG.info("Using HoodieRealtimeInputFormat");
|
LOG.info("Using HoodieRealtimeInputFormat");
|
||||||
input = new HoodieParquetRealtimeInputFormat();
|
input = createParquetRealtimeInputFormat();
|
||||||
} else {
|
} else {
|
||||||
LOG.info("Using HoodieInputFormat");
|
LOG.info("Using HoodieInputFormat");
|
||||||
input = new HoodieParquetInputFormat();
|
input = createParquetInputFormat();
|
||||||
}
|
}
|
||||||
input.setConf(job.getConfiguration());
|
input.setConf(job.getConfiguration());
|
||||||
result = new ArrayList<>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
|
result = new ArrayList<>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
|
||||||
@@ -901,7 +917,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
|||||||
job.set("hudi.hive.realtime", "true");
|
job.set("hudi.hive.realtime", "true");
|
||||||
InputSplit[] splits;
|
InputSplit[] splits;
|
||||||
if (hoodieFilter) {
|
if (hoodieFilter) {
|
||||||
HoodieParquetInputFormat input = new HoodieParquetRealtimeInputFormat();
|
HoodieParquetInputFormat input = createParquetRealtimeInputFormat();
|
||||||
input.setConf(job);
|
input.setConf(job);
|
||||||
splits = input.getSplits(job, numSplits);
|
splits = input.getSplits(job, numSplits);
|
||||||
} else {
|
} else {
|
||||||
|
|||||||
Reference in New Issue
Block a user