Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-9970

HBase BulkLoad, table is creating with the timestamp key also as a column to the table.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.94.11
    • 0.98.0, 0.96.1, 0.94.14
    • None
    • None
    • Reviewed

    Description

      If BulkLoad job is running with out creating a table.
      job itself will create the table if table is not found.

      if (!doesTableExist(tableName)) {
              createTable(conf, tableName);
            }
      

      if columns contains timestamp also then table is creating with defined columns and timestamp key.

      eg: -Dimporttsv.columns=HBASE_ROW_KEY,HBASE_TS_KEY,d:num

      table is creating with the following columnFamilies.
      'HBASE_TS_KEY' and 'd'

      while iterating timestamp key also need to avoid while describing the column descriptors.

      private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
            throws IOException {
          HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
          Set<String> cfSet = new HashSet<String>();
          for (String aColumn : columns) {
            if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)) continue;
            // we are only concerned with the first one (in case this is a cf:cq)
            cfSet.add(aColumn.split(":", 2)[0]);
          }
          for (String cf : cfSet) {
            HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
            htd.addFamily(hcd);
          }
          LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
            tableName, cfSet));
          admin.createTable(htd);
        }
      
      Index: hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
      ===================================================================
      --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java	(revision 1539967)
      +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java	(working copy)
      @@ -413,7 +413,8 @@
           HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
           Set<String> cfSet = new HashSet<String>();
           for (String aColumn : columns) {
      -      if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)) continue;
      +      if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
      +          || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)) continue;
             // we are only concerned with the first one (in case this is a cf:cq)
             cfSet.add(aColumn.split(":", 2)[0]);
           }
      
      

      Attachments

        1. HBASE-9970_94.patch
          0.8 kB
          Y. SREENIVASULU REDDY
        2. HBASE-9970.java.patch
          0.8 kB
          Y. SREENIVASULU REDDY

        Activity

          People

            sreenivasulureddy Y. SREENIVASULU REDDY
            sreenivasulureddy Y. SREENIVASULU REDDY
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: