Y. SREENIVASULU REDDY created HBASE-9970: --------------------------------------------
Summary: HBase BulkLoad, table is creating with the timestamp key also as a column to the table. Key: HBASE-9970 URL: https://issues.apache.org/jira/browse/HBASE-9970 Project: HBase Issue Type: Bug Affects Versions: 0.94.11 Reporter: Y. SREENIVASULU REDDY Assignee: Y. SREENIVASULU REDDY Fix For: 0.98.0, 0.96.1, 0.94.14 If BulkLoad job is running with out creating a table. job itself will create the table if table is not found. {quote} if (!doesTableExist(tableName)) { createTable(conf, tableName); } {quote} if columns contains timestamp also then table is creating with defined columns and timestamp key. {quote} eg: -Dimporttsv.columns=HBASE_ROW_KEY,HBASE_TS_KEY,d:num {quote} 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. {code} 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); } {code} {quote} 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]); } {quote} -- This message was sent by Atlassian JIRA (v6.1#6144)