-
Notifications
You must be signed in to change notification settings - Fork 345
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
02fa2d3
commit 9b40ba8
Showing
5 changed files
with
214 additions
and
17 deletions.
There are no files selected for viewing
86 changes: 86 additions & 0 deletions
86
...lates/cdap-etl/cdap-etl-core/src/main/java/io/cdap/cdap/etl/batch/WrappedInputFormat.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* Copyright © 2024 Cask Data, Inc. | ||
* | ||
* Licensed 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 io.cdap.cdap.etl.batch; | ||
|
||
import io.cdap.cdap.api.exception.WrappedException; | ||
import java.io.IOException; | ||
import java.util.List; | ||
import org.apache.hadoop.mapreduce.InputFormat; | ||
import org.apache.hadoop.mapreduce.InputSplit; | ||
import org.apache.hadoop.mapreduce.JobContext; | ||
import org.apache.hadoop.mapreduce.RecordReader; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
|
||
public class WrappedInputFormat<K, V> extends InputFormat<K, V> { | ||
private final InputFormat<K, V> inputFormat; | ||
private final String stageName; | ||
|
||
@Override | ||
public List<InputSplit> getSplits(JobContext jobContext) | ||
throws IOException, InterruptedException { | ||
try { | ||
return inputFormat.getSplits(jobContext); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public RecordReader<K, V> createRecordReader(InputSplit inputSplit, | ||
TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { | ||
try { | ||
return new WrappedRecordReader<>(inputFormat.createRecordReader(inputSplit, | ||
taskAttemptContext), stageName); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
/** | ||
* Returns the delegating {@link InputFormat} based on the current configuration. | ||
* | ||
* @param classLoader the {@link ClassLoader} for loading input format | ||
* @param inputFormatClassName the name of {@link InputFormat} class | ||
* @throws IOException if failed to instantiate the input format class | ||
*/ | ||
public WrappedInputFormat(ClassLoader classLoader, String inputFormatClassName, | ||
String stageName) throws IOException { | ||
this.stageName = stageName; | ||
if (inputFormatClassName == null) { | ||
throw new IllegalArgumentException("Missing configuration for the InputFormat to use"); | ||
} | ||
if (inputFormatClassName.equals(getClass().getName())) { | ||
throw new IllegalArgumentException("Cannot delegate InputFormat to the same class"); | ||
} | ||
try { | ||
//noinspection unchecked | ||
@SuppressWarnings("unchecked") | ||
Class<InputFormat<K, V>> inputFormatClass = (Class<InputFormat<K, V>>) classLoader.loadClass( | ||
inputFormatClassName); | ||
this.inputFormat = inputFormatClass.newInstance(); | ||
} catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) { | ||
throw new IOException( | ||
String.format("Unable to instantiate delegate input format %s", inputFormatClassName), e); | ||
} | ||
} | ||
} |
106 changes: 106 additions & 0 deletions
106
...ates/cdap-etl/cdap-etl-core/src/main/java/io/cdap/cdap/etl/batch/WrappedRecordReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,106 @@ | ||
/* | ||
* Copyright © 2024 Cask Data, Inc. | ||
* | ||
* Licensed 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 io.cdap.cdap.etl.batch; | ||
|
||
import io.cdap.cdap.api.exception.WrappedException; | ||
import java.io.IOException; | ||
import org.apache.hadoop.mapreduce.InputSplit; | ||
import org.apache.hadoop.mapreduce.RecordReader; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
|
||
public class WrappedRecordReader<K, V> extends RecordReader<K, V> { | ||
private final RecordReader<K, V> recordReader; | ||
private final String stageName; | ||
|
||
public WrappedRecordReader(RecordReader<K, V> recordReader, String stageName) { | ||
this.recordReader = recordReader; | ||
this.stageName = stageName; | ||
} | ||
|
||
@Override | ||
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) | ||
throws IOException, InterruptedException { | ||
try { | ||
recordReader.initialize(inputSplit, taskAttemptContext); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public boolean nextKeyValue() throws IOException, InterruptedException { | ||
try { | ||
return recordReader.nextKeyValue(); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public K getCurrentKey() throws IOException, InterruptedException { | ||
try { | ||
return recordReader.getCurrentKey(); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public V getCurrentValue() throws IOException, InterruptedException { | ||
try { | ||
return recordReader.getCurrentValue(); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public float getProgress() throws IOException, InterruptedException { | ||
try { | ||
return recordReader.getProgress(); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
|
||
@Override | ||
public void close() throws IOException { | ||
try { | ||
recordReader.close(); | ||
} catch (Exception e) { | ||
if (stageName != null) { | ||
throw new WrappedException(e, stageName); | ||
} | ||
throw e; | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters