Hi Flink devs, this is my final report about the HBaseOutputFormat problem (with Flink 0.8.1) and I hope you could suggest me the best way to make a PR:
1) The following code produce the error reported below (this should be fixed in 0.9 right?) Job job = Job.getInstance(); myDataset.output( new HadoopOutputFormat<Text, *Mutation*>(new *TableOutputFormat*<Text>(), job)); org.apache.flink.api.common.functions.InvalidTypesException: Interfaces and abstract classes are not valid types: class org.apache.hadoop.hbase.client.Mutation at org.apache.flink.api.java.typeutils.TypeExtractor.privateGetForClass(TypeExtractor.java:885) at org.apache.flink.api.java.typeutils.TypeExtractor.privateGetForClass(TypeExtractor.java:877) at org.apache.flink.api.java.typeutils.TypeExtractor.createTypeInfoWithTypeHierarchy(TypeExtractor.java:376) at org.apache.flink.api.java.typeutils.TypeExtractor.createTypeInfoWithTypeHierarchy(TypeExtractor.java:296) at org.apache.flink.api.java.typeutils.TypeExtractor.privateCreateTypeInfo(TypeExtractor.java:224) at org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:152) at org.apache.flink.api.java.typeutils.TypeExtractor.getMapReturnTypes(TypeExtractor.java:79) at org.apache.flink.api.java.DataSet.map(DataSet.java:160) 2) So I created a custom HBaseTableOutputFormat -*see at the end of the mail-* (that is basically copied from to the HBase TableInputFormat) that sets correctly the "mapred.output.dir" param required by the HadoopOutputFormatBase so I can make it work: Job job = Job.getInstance(); job.getConfiguration().set(TableOutputFormat.OUTPUT_TABLE, outputTableName); HBaseTableOutputFormat<Text> hbaseTOF = new HBaseTableOutputFormat<>(); HadoopOutputFormat<Text, Put> outOF = new HadoopOutputFormat<>(hbaseTOF, job); myDataset.output(outOF); 3) However this does still not work unless you call setConf() of Configurable subclasses in the HadoopOutputFormatBase: - in the* public void finalizeGlobal(int parallelism) throws IOException* method: .... * if(this.mapreduceOutputFormat instanceof Configurable){* * ((Configurable)this.mapreduceOutputFormat).setConf(this.configuration);* * }* this.fileOutputCommitter = new FileOutputCommitter(new Path(this.configuration.get("mapred.output.dir")), taskContext); .... - In the* public void open(int taskNumber, int numTasks) throws IOException* method: .... * if(this.mapreduceOutputFormat instanceof Configurable){* * ((Configurable)this.mapreduceOutputFormat).setConf(this.configuration);* * }* try { this.context = HadoopUtils.instantiateTaskAttemptContext(this.configuration, taskAttemptID); } catch (Exception e) { throw new RuntimeException(e); } .... 4) Probably the modifications apported in point 3 should be applied both for mapreduce and mapred packages.. Thanks in advace, Flavio ----------------------------------------------------------------------- this is the HadoopOutputFormatBase.java: ----------------------------------------------------------------------- import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.mapreduce.TableOutputCommitter; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.OutputCommitter; import org.apache.hadoop.mapreduce.OutputFormat; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; /** * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored * while the output value <u>must</u> be either a {@link Put} or a * {@link Delete} instance. * * @param <KEY> The type of the key. Ignored in this class. */ @InterfaceAudience.Public @InterfaceStability.Stable public class HBaseTableOutputFormat<KEY>* extends OutputFormat<KEY, Put>* implements Configurable { private final Log LOG = LogFactory.getLog(HBaseTableOutputFormat.class); /** Job parameter that specifies the output table. */ public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; /** * Optional job parameter to specify a peer cluster. * Used specifying remote cluster when copying between hbase clusters (the * source is picked up from <code>hbase-site.xml</code>). * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, Class, String, String, String) */ public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum"; /** Optional job parameter to specify peer cluster's ZK client port */ public static final String QUORUM_PORT = "hbase.mapred.output.quorum.port"; /** Optional specification of the rs class name of the peer cluster */ public static final String REGION_SERVER_CLASS = "hbase.mapred.output.rs.class"; /** Optional specification of the rs impl name of the peer cluster */ public static final String REGION_SERVER_IMPL = "hbase.mapred.output.rs.impl"; /** The configuration. */ private Configuration conf = null; private HTable table; /** * Writes the reducer output to an HBase table. * * @param <KEY> The type of the key. */ protected static class TableRecordWriter<KEY> *extends RecordWriter<KEY, Put> *{ /** The table to write to. */ private HTable table; /** * Instantiate a TableRecordWriter with the HBase HClient for writing. * * @param table The table to write to. */ public TableRecordWriter(HTable table) { this.table = table; } /** * Closes the writer, in this case flush table commits. * * @param context The context. * @throws IOException When closing the writer fails. * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext) */ @Override public void close(TaskAttemptContext context) throws IOException { table.close(); } /** * Writes a key/value pair into the table. * * @param key The key. * @param value The value. * @throws IOException When writing fails. * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object) */ @Override *public void write(KEY key, Put value)* * throws IOException {* * if (value instanceof Put) this.table.put(new Put((Put)value));* *// else if (value instanceof Delete) this.table.delete(new Delete((Delete)value));* * else throw new IOException("Pass a Delete or a Put");* * }* } /** * Creates a new record writer. * * @param context The current task context. * @return The newly created writer instance. * @throws IOException When creating the writer fails. * @throws InterruptedException When the jobs is cancelled. * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext) */ @Override public RecordWriter<KEY, *Put*> getRecordWriter( TaskAttemptContext context) throws IOException, InterruptedException { return new TableRecordWriter<KEY>(this.table); } /** * Checks if the output target exists. * * @param context The current context. * @throws IOException When the check fails. * @throws InterruptedException When the job is aborted. * @see org.apache.hadoop.mapreduce.OutputFormat#checkOutputSpecs(org.apache.hadoop.mapreduce.JobContext) */ @Override public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { // TODO Check if the table exists? } /** * Returns the output committer. * * @param context The current context. * @return The committer. * @throws IOException When creating the committer fails. * @throws InterruptedException When the job is aborted. * @see org.apache.hadoop.mapreduce.OutputFormat#getOutputCommitter(org.apache.hadoop.mapreduce.TaskAttemptContext) */ @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException { return new TableOutputCommitter(); } public Configuration getConf() { return conf; } @Override public void setConf(Configuration otherConf) { this.conf = HBaseConfiguration.create(otherConf); String tableName = this.conf.get(OUTPUT_TABLE); if(tableName == null || tableName.length() <= 0) { throw new IllegalArgumentException("Must specify table name"); } String address = this.conf.get(QUORUM_ADDRESS); int zkClientPort = this.conf.getInt(QUORUM_PORT, 0); String serverClass = this.conf.get(REGION_SERVER_CLASS); String serverImpl = this.conf.get(REGION_SERVER_IMPL); try { if (address != null) { ZKUtil.applyClusterKeyToConf(this.conf, address); } if (serverClass != null) { this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl); } if (zkClientPort != 0) { this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); } this.table = new HTable(this.conf, tableName); this.table.setAutoFlush(false, true); * String outDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), this.table.getName()).toString();* * this.conf.set("mapred.output.dir", outDir);* * otherConf.set("mapred.output.dir", outDir);* LOG.info("Created table instance for " + tableName); } catch(IOException e) { LOG.error(e); throw new RuntimeException(e); } } }