001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.mapred;
019
020import java.io.IOException;
021import org.apache.hadoop.fs.Path;
022import org.apache.hadoop.hbase.TableName;
023import org.apache.hadoop.hbase.client.Connection;
024import org.apache.hadoop.hbase.client.ConnectionFactory;
025import org.apache.hadoop.hbase.util.Bytes;
026import org.apache.hadoop.mapred.FileInputFormat;
027import org.apache.hadoop.mapred.JobConf;
028import org.apache.hadoop.mapred.JobConfigurable;
029import org.apache.hadoop.util.StringUtils;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.slf4j.Logger;
032import org.slf4j.LoggerFactory;
033
034/**
035 * Convert HBase tabular data into a format that is consumable by Map/Reduce.
036 */
037@InterfaceAudience.Public
038public class TableInputFormat extends TableInputFormatBase implements JobConfigurable {
039  private static final Logger LOG = LoggerFactory.getLogger(TableInputFormat.class);
040
041  /**
042   * space delimited list of columns
043   */
044  public static final String COLUMN_LIST = "hbase.mapred.tablecolumns";
045
046  public void configure(JobConf job) {
047    try {
048      initialize(job);
049    } catch (Exception e) {
050      LOG.error(StringUtils.stringifyException(e));
051    }
052  }
053
054  @Override
055  protected void initialize(JobConf job) throws IOException {
056    Path[] tableNames = FileInputFormat.getInputPaths(job);
057    String colArg = job.get(COLUMN_LIST);
058    String[] colNames = colArg.split(" ");
059    byte[][] m_cols = new byte[colNames.length][];
060    for (int i = 0; i < m_cols.length; i++) {
061      m_cols[i] = Bytes.toBytes(colNames[i]);
062    }
063    setInputColumns(m_cols);
064    Connection connection = ConnectionFactory.createConnection(job);
065    initializeTable(connection, TableName.valueOf(tableNames[0].getName()));
066  }
067
068  public void validateInput(JobConf job) throws IOException {
069    // expecting exactly one path
070    Path[] tableNames = FileInputFormat.getInputPaths(job);
071    if (tableNames == null || tableNames.length > 1) {
072      throw new IOException("expecting one table name");
073    }
074
075    // connected to table?
076    if (getTable() == null) {
077      throw new IOException("could not connect to table '" + tableNames[0].getName() + "'");
078    }
079
080    // expecting at least one column
081    String colArg = job.get(COLUMN_LIST);
082    if (colArg == null || colArg.length() == 0) {
083      throw new IOException("expecting at least one column");
084    }
085  }
086}