blob: 7d9d74f6c123859334288de571b5598fb5ce85a9 [file] [log] [blame]
/*
* 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.hcatalog.mapreduce;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.TreeMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.metastore.Warehouse;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.metastore.api.SerDeInfo;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.OutputFormat;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hcatalog.common.ErrorType;
import org.apache.hcatalog.common.HCatConstants;
import org.apache.hcatalog.common.HCatException;
import org.apache.hcatalog.common.HCatUtil;
import org.apache.hcatalog.data.HCatRecord;
import org.apache.hcatalog.data.schema.HCatFieldSchema;
import org.apache.hcatalog.data.schema.HCatSchema;
import org.apache.hcatalog.data.schema.HCatSchemaUtils;
import org.apache.hcatalog.rcfile.RCFileInputDriver;
import org.apache.hcatalog.rcfile.RCFileOutputDriver;
/**
* The OutputFormat to use to write data to HCat without a hcat server. This can then
* be imported into a hcat instance, or used with a HCatEximInputFormat. As in
* HCatOutputFormat, the key value is ignored and
* and should be given as null. The value is the HCatRecord to write.
*/
public class HCatEximOutputFormat extends HCatBaseOutputFormat {
private static final Log LOG = LogFactory.getLog(HCatEximOutputFormat.class);
/**
* Get the record writer for the job. Uses the Table's default OutputStorageDriver
* to get the record writer.
*
* @param context
* the information about the current task.
* @return a RecordWriter to write the output for the job.
* @throws IOException
*/
@Override
public RecordWriter<WritableComparable<?>, HCatRecord>
getRecordWriter(TaskAttemptContext context
) throws IOException, InterruptedException {
HCatRecordWriter rw = new HCatRecordWriter(context);
return rw;
}
/**
* Get the output committer for this output format. This is responsible
* for ensuring the output is committed correctly.
* @param context the task context
* @return an output committer
* @throws IOException
* @throws InterruptedException
*/
@Override
public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException {
OutputFormat<? super WritableComparable<?>, ? super Writable> outputFormat = getOutputFormat(context);
return new HCatEximOutputCommitter(context,outputFormat.getOutputCommitter(context));
}
public static void setOutput(Job job, String dbname, String tablename, String location,
HCatSchema partitionSchema, List<String> partitionValues, HCatSchema columnSchema) throws HCatException {
setOutput(job, dbname, tablename, location, partitionSchema, partitionValues, columnSchema,
RCFileInputDriver.class.getName(),
RCFileOutputDriver.class.getName(),
RCFileInputFormat.class.getName(),
RCFileOutputFormat.class.getName(),
ColumnarSerDe.class.getName());
}
@SuppressWarnings("unchecked")
public static void setOutput(Job job, String dbname, String tablename, String location,
HCatSchema partitionSchema,
List<String> partitionValues,
HCatSchema columnSchema,
String isdname, String osdname,
String ifname, String ofname,
String serializationLib) throws HCatException {
Map<String, String> partSpec = new TreeMap<String, String>();
List<HCatFieldSchema> partKeys = null;
if (partitionSchema != null) {
partKeys = partitionSchema.getFields();
if (partKeys.size() != partitionValues.size()) {
throw new IllegalArgumentException("Partition key size differs from partition value size");
}
for (int i = 0; i < partKeys.size(); ++i) {
HCatFieldSchema partKey = partKeys.get(i);
if (partKey.getType() != HCatFieldSchema.Type.STRING) {
throw new IllegalArgumentException("Partition key type string is only supported");
}
partSpec.put(partKey.getName(), partitionValues.get(i));
}
}
StorerInfo storerInfo = new StorerInfo(isdname, osdname, new Properties());
HCatTableInfo outputInfo = HCatTableInfo.getOutputTableInfo(null, null, dbname, tablename,
partSpec);
org.apache.hadoop.hive.ql.metadata.Table tbl = new
org.apache.hadoop.hive.ql.metadata.Table(dbname, tablename);
Table table = tbl.getTTable();
table.getParameters().put(HCatConstants.HCAT_ISD_CLASS, isdname);
table.getParameters().put(HCatConstants.HCAT_OSD_CLASS, osdname);
try {
String partname = null;
if ((partKeys != null) && !partKeys.isEmpty()) {
List<FieldSchema> partSchema = HCatSchemaUtils.getFieldSchemas(partKeys);
table.setPartitionKeys(partSchema);
partname = Warehouse.makePartName(partSchema, partitionValues);
} else {
partname = "data";
}
StorageDescriptor sd = table.getSd();
sd.setLocation(location);
String dataLocation = location + "/" + partname;
OutputJobInfo jobInfo = new OutputJobInfo(outputInfo,
columnSchema, columnSchema, storerInfo, dataLocation, table);
setPartDetails(jobInfo, columnSchema, partSpec);
sd.setCols(HCatUtil.getFieldSchemaList(jobInfo.getOutputSchema().getFields()));
sd.setInputFormat(ifname);
sd.setOutputFormat(ofname);
SerDeInfo serdeInfo = sd.getSerdeInfo();
serdeInfo.setSerializationLib(serializationLib);
Configuration conf = job.getConfiguration();
conf.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(jobInfo));
} catch (IOException e) {
throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e);
} catch (MetaException e) {
throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e);
}
}
}