-
Notifications
You must be signed in to change notification settings - Fork 4.6k
/
DynamicPartitionFileRecordWriterContainer.java
238 lines (210 loc) · 10.8 KB
/
DynamicPartitionFileRecordWriterContainer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
/**
* 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.hive.hcatalog.mapreduce;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.serde2.AbstractSerDe;
import org.apache.hadoop.hive.serde2.SerDeException;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.RecordWriter;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hive.hcatalog.common.ErrorType;
import org.apache.hive.hcatalog.common.HCatException;
import org.apache.hive.hcatalog.data.HCatRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Record writer container for tables using dynamic partitioning. See
* {@link FileOutputFormatContainer} for more information
*/
class DynamicPartitionFileRecordWriterContainer extends FileRecordWriterContainer {
private static final Logger LOG = LoggerFactory.getLogger(DynamicPartitionFileRecordWriterContainer.class);
private final List<Integer> dynamicPartCols;
private int maxDynamicPartitions;
private final Map<String, RecordWriter<? super WritableComparable<?>, ? super Writable>> baseDynamicWriters;
private final Map<String, AbstractSerDe> baseDynamicSerDe;
private final Map<String, org.apache.hadoop.mapred.OutputCommitter> baseDynamicCommitters;
private final Map<String, org.apache.hadoop.mapred.TaskAttemptContext> dynamicContexts;
private final Map<String, ObjectInspector> dynamicObjectInspectors;
private Map<String, OutputJobInfo> dynamicOutputJobInfo;
private String HIVE_DEFAULT_PARTITION_VALUE = null;
/**
* @param baseWriter RecordWriter to contain
* @param context current TaskAttemptContext
* @throws IOException
* @throws InterruptedException
*/
public DynamicPartitionFileRecordWriterContainer(
RecordWriter<? super WritableComparable<?>, ? super Writable> baseWriter,
TaskAttemptContext context) throws IOException, InterruptedException {
super(baseWriter, context);
maxDynamicPartitions = jobInfo.getMaxDynamicPartitions();
dynamicPartCols = jobInfo.getPosOfDynPartCols();
if (dynamicPartCols == null) {
throw new HCatException("It seems that setSchema() is not called on "
+ "HCatOutputFormat. Please make sure that method is called.");
}
this.baseDynamicSerDe = new HashMap<String, AbstractSerDe>();
this.baseDynamicWriters =
new HashMap<String, RecordWriter<? super WritableComparable<?>, ? super Writable>>();
this.baseDynamicCommitters = new HashMap<String, org.apache.hadoop.mapred.OutputCommitter>();
this.dynamicContexts = new HashMap<String, org.apache.hadoop.mapred.TaskAttemptContext>();
this.dynamicObjectInspectors = new HashMap<String, ObjectInspector>();
this.dynamicOutputJobInfo = new HashMap<String, OutputJobInfo>();
this.HIVE_DEFAULT_PARTITION_VALUE = HiveConf.getVar(context.getConfiguration(), HiveConf.ConfVars.DEFAULTPARTITIONNAME);
}
@Override
public void close(TaskAttemptContext context) throws IOException, InterruptedException {
Reporter reporter = InternalUtil.createReporter(context);
for (RecordWriter<? super WritableComparable<?>, ? super Writable> bwriter : baseDynamicWriters
.values()) {
// We are in RecordWriter.close() make sense that the context would be
// TaskInputOutput.
bwriter.close(reporter);
}
TaskCommitContextRegistry.getInstance().register(context, new TaskCommitContextRegistry.TaskCommitterProxy() {
@Override
public void abortTask(TaskAttemptContext context) throws IOException {
for (Map.Entry<String, OutputJobInfo> outputJobInfoEntry : dynamicOutputJobInfo.entrySet()) {
String dynKey = outputJobInfoEntry.getKey();
OutputJobInfo outputJobInfo = outputJobInfoEntry.getValue();
LOG.info("Aborting task-attempt for " + outputJobInfo.getLocation());
baseDynamicCommitters.get(dynKey)
.abortTask(dynamicContexts.get(dynKey));
}
}
@Override
public void commitTask(TaskAttemptContext context) throws IOException {
for (Map.Entry<String, OutputJobInfo> outputJobInfoEntry : dynamicOutputJobInfo.entrySet()) {
String dynKey = outputJobInfoEntry.getKey();
OutputJobInfo outputJobInfo = outputJobInfoEntry.getValue();
LOG.info("Committing task-attempt for " + outputJobInfo.getLocation());
TaskAttemptContext dynContext = dynamicContexts.get(dynKey);
OutputCommitter dynCommitter = baseDynamicCommitters.get(dynKey);
if (dynCommitter.needsTaskCommit(dynContext)) {
dynCommitter.commitTask(dynContext);
}
else {
LOG.info("Skipping commitTask() for " + outputJobInfo.getLocation());
}
}
}
});
}
@Override
protected LocalFileWriter getLocalFileWriter(HCatRecord value) throws IOException, HCatException {
OutputJobInfo localJobInfo = null;
// Calculate which writer to use from the remaining values - this needs to
// be done before we delete cols.
List<String> dynamicPartValues = new ArrayList<String>();
for (Integer colToAppend : dynamicPartCols) {
Object partitionValue = value.get(colToAppend);
dynamicPartValues.add(partitionValue == null? HIVE_DEFAULT_PARTITION_VALUE : partitionValue.toString());
}
String dynKey = dynamicPartValues.toString();
if (!baseDynamicWriters.containsKey(dynKey)) {
if ((maxDynamicPartitions != -1) && (baseDynamicWriters.size() > maxDynamicPartitions)) {
throw new HCatException(ErrorType.ERROR_TOO_MANY_DYNAMIC_PTNS,
"Number of dynamic partitions being created "
+ "exceeds configured max allowable partitions[" + maxDynamicPartitions
+ "], increase parameter [" + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname
+ "] if needed.");
}
org.apache.hadoop.mapred.TaskAttemptContext currTaskContext =
HCatMapRedUtil.createTaskAttemptContext(context);
configureDynamicStorageHandler(currTaskContext, dynamicPartValues);
localJobInfo = HCatBaseOutputFormat.getJobInfo(currTaskContext.getConfiguration());
// Setup serDe.
AbstractSerDe currSerDe =
ReflectionUtils.newInstance(storageHandler.getSerDeClass(), currTaskContext.getJobConf());
try {
InternalUtil.initializeOutputSerDe(currSerDe, currTaskContext.getConfiguration(),
localJobInfo);
} catch (SerDeException e) {
throw new IOException("Failed to initialize SerDe", e);
}
// create base OutputFormat
org.apache.hadoop.mapred.OutputFormat baseOF =
ReflectionUtils.newInstance(storageHandler.getOutputFormatClass(),
currTaskContext.getJobConf());
// We are skipping calling checkOutputSpecs() for each partition
// As it can throw a FileAlreadyExistsException when more than one
// mapper is writing to a partition.
// See HCATALOG-490, also to avoid contacting the namenode for each new
// FileOutputFormat instance.
// In general this should be ok for most FileOutputFormat implementations
// but may become an issue for cases when the method is used to perform
// other setup tasks.
// Get Output Committer
org.apache.hadoop.mapred.OutputCommitter baseOutputCommitter =
currTaskContext.getJobConf().getOutputCommitter();
// Create currJobContext the latest so it gets all the config changes
org.apache.hadoop.mapred.JobContext currJobContext =
HCatMapRedUtil.createJobContext(currTaskContext);
// Set up job.
baseOutputCommitter.setupJob(currJobContext);
// Recreate to refresh jobConf of currTask context.
currTaskContext =
HCatMapRedUtil.createTaskAttemptContext(currJobContext.getJobConf(),
currTaskContext.getTaskAttemptID(), currTaskContext.getProgressible());
// Set temp location.
currTaskContext.getConfiguration().set(
"mapred.work.output.dir",
new FileOutputCommitter(new Path(localJobInfo.getLocation()), currTaskContext)
.getWorkPath().toString());
// Set up task.
baseOutputCommitter.setupTask(currTaskContext);
Path parentDir = new Path(currTaskContext.getConfiguration().get("mapred.work.output.dir"));
Path childPath =
new Path(parentDir, FileOutputFormat.getUniqueFile(currTaskContext,
currTaskContext.getConfiguration().get("mapreduce.output.basename", "part"), ""));
RecordWriter baseRecordWriter =
baseOF.getRecordWriter(parentDir.getFileSystem(currTaskContext.getConfiguration()),
currTaskContext.getJobConf(), childPath.toString(),
InternalUtil.createReporter(currTaskContext));
baseDynamicWriters.put(dynKey, baseRecordWriter);
baseDynamicSerDe.put(dynKey, currSerDe);
baseDynamicCommitters.put(dynKey, baseOutputCommitter);
dynamicContexts.put(dynKey, currTaskContext);
dynamicObjectInspectors.put(dynKey,
InternalUtil.createStructObjectInspector(jobInfo.getOutputSchema()));
dynamicOutputJobInfo.put(dynKey,
HCatOutputFormat.getJobInfo(dynamicContexts.get(dynKey).getConfiguration()));
}
return new LocalFileWriter(baseDynamicWriters.get(dynKey), dynamicObjectInspectors.get(dynKey),
baseDynamicSerDe.get(dynKey), dynamicOutputJobInfo.get(dynKey));
}
protected void configureDynamicStorageHandler(JobContext context, List<String> dynamicPartVals)
throws IOException {
HCatOutputFormat.configureOutputStorageHandler(context, dynamicPartVals);
}
}