private void addNewColumn( String databaseName, String tableName, CatalogProtos.ColumnProto columnProto) { HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null; try { client = clientPool.getClient(); Table table = client.getHiveClient().getTable(databaseName, tableName); List<FieldSchema> columns = table.getSd().getCols(); columns.add( new FieldSchema( columnProto.getName(), HiveCatalogUtil.getHiveFieldType(columnProto.getDataType()), "")); client.getHiveClient().alter_table(databaseName, tableName, table); } catch (NoSuchObjectException nsoe) { } catch (Exception e) { throw new TajoInternalError(e); } finally { if (client != null) { client.release(); } } }
@Override public final void createTable(final CatalogProtos.TableDescProto tableDescProto) throws CatalogException { HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null; TableDesc tableDesc = new TableDesc(tableDescProto); String[] splitted = CatalogUtil.splitFQTableName(tableDesc.getName()); String databaseName = splitted[0]; String tableName = splitted[1]; try { client = clientPool.getClient(); org.apache.hadoop.hive.metastore.api.Table table = new org.apache.hadoop.hive.metastore.api.Table(); table.setDbName(databaseName); table.setTableName(tableName); table.setParameters( new HashMap<String, String>(tableDesc.getMeta().getOptions().getAllKeyValus())); // TODO: set owner // table.setOwner(); StorageDescriptor sd = new StorageDescriptor(); sd.setSerdeInfo(new SerDeInfo()); sd.getSerdeInfo().setParameters(new HashMap<String, String>()); sd.getSerdeInfo().setName(table.getTableName()); // if tajo set location method, thrift client make exception as follows: // Caused by: MetaException(message:java.lang.NullPointerException) // If you want to modify table path, you have to modify on Hive cli. if (tableDesc.isExternal()) { table.setTableType(TableType.EXTERNAL_TABLE.name()); table.putToParameters("EXTERNAL", "TRUE"); Path tablePath = new Path(tableDesc.getUri()); FileSystem fs = tablePath.getFileSystem(conf); if (fs.isFile(tablePath)) { LOG.warn("A table path is a file, but HiveCatalogStore does not allow a file path."); sd.setLocation(tablePath.getParent().toString()); } else { sd.setLocation(tablePath.toString()); } } // set column information List<Column> columns = tableDesc.getSchema().getRootColumns(); ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(columns.size()); for (Column eachField : columns) { cols.add( new FieldSchema( eachField.getSimpleName(), HiveCatalogUtil.getHiveFieldType(eachField.getDataType()), "")); } sd.setCols(cols); // set partition keys if (tableDesc.hasPartition() && tableDesc.getPartitionMethod().getPartitionType().equals(PartitionType.COLUMN)) { List<FieldSchema> partitionKeys = new ArrayList<FieldSchema>(); for (Column eachPartitionKey : tableDesc.getPartitionMethod().getExpressionSchema().getRootColumns()) { partitionKeys.add( new FieldSchema( eachPartitionKey.getSimpleName(), HiveCatalogUtil.getHiveFieldType(eachPartitionKey.getDataType()), "")); } table.setPartitionKeys(partitionKeys); } if (tableDesc.getMeta().getStoreType().equalsIgnoreCase(BuiltinStorages.RCFILE)) { String serde = tableDesc.getMeta().getOption(StorageConstants.RCFILE_SERDE); sd.setInputFormat(org.apache.hadoop.hive.ql.io.RCFileInputFormat.class.getName()); sd.setOutputFormat(org.apache.hadoop.hive.ql.io.RCFileOutputFormat.class.getName()); if (StorageConstants.DEFAULT_TEXT_SERDE.equals(serde)) { sd.getSerdeInfo() .setSerializationLib( org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe.class.getName()); } else { sd.getSerdeInfo() .setSerializationLib( org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe.class.getName()); } if (tableDesc.getMeta().getOptions().containsKey(StorageConstants.RCFILE_NULL)) { table.putToParameters( serdeConstants.SERIALIZATION_NULL_FORMAT, StringEscapeUtils.unescapeJava( tableDesc.getMeta().getOption(StorageConstants.RCFILE_NULL))); } } else if (tableDesc.getMeta().getStoreType().equals(BuiltinStorages.TEXT)) { sd.getSerdeInfo() .setSerializationLib( org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName()); sd.setInputFormat(org.apache.hadoop.mapred.TextInputFormat.class.getName()); sd.setOutputFormat( org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat.class.getName()); String fieldDelimiter = tableDesc .getMeta() .getOption( StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); // User can use an unicode for filed delimiter such as \u0001, \001. // In this case, java console will convert this value into "\\u001". // And hive will un-espace this value again. // As a result, user can use right field delimiter. // So, we have to un-escape this value. sd.getSerdeInfo() .putToParameters( serdeConstants.SERIALIZATION_FORMAT, StringEscapeUtils.unescapeJava(fieldDelimiter)); sd.getSerdeInfo() .putToParameters( serdeConstants.FIELD_DELIM, StringEscapeUtils.unescapeJava(fieldDelimiter)); table.getParameters().remove(StorageConstants.TEXT_DELIMITER); if (tableDesc.getMeta().containsOption(StorageConstants.TEXT_NULL)) { table.putToParameters( serdeConstants.SERIALIZATION_NULL_FORMAT, StringEscapeUtils.unescapeJava( tableDesc.getMeta().getOption(StorageConstants.TEXT_NULL))); table.getParameters().remove(StorageConstants.TEXT_NULL); } } else if (tableDesc .getMeta() .getStoreType() .equalsIgnoreCase(BuiltinStorages.SEQUENCE_FILE)) { String serde = tableDesc.getMeta().getOption(StorageConstants.SEQUENCEFILE_SERDE); sd.setInputFormat(org.apache.hadoop.mapred.SequenceFileInputFormat.class.getName()); sd.setOutputFormat( org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat.class.getName()); if (StorageConstants.DEFAULT_TEXT_SERDE.equals(serde)) { sd.getSerdeInfo() .setSerializationLib( org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName()); String fieldDelimiter = tableDesc .getMeta() .getOption( StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); // User can use an unicode for filed delimiter such as \u0001, \001. // In this case, java console will convert this value into "\\u001". // And hive will un-espace this value again. // As a result, user can use right field delimiter. // So, we have to un-escape this value. sd.getSerdeInfo() .putToParameters( serdeConstants.SERIALIZATION_FORMAT, StringEscapeUtils.unescapeJava(fieldDelimiter)); sd.getSerdeInfo() .putToParameters( serdeConstants.FIELD_DELIM, StringEscapeUtils.unescapeJava(fieldDelimiter)); table.getParameters().remove(StorageConstants.SEQUENCEFILE_DELIMITER); } else { sd.getSerdeInfo() .setSerializationLib( org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe.class.getName()); } if (tableDesc.getMeta().containsOption(StorageConstants.SEQUENCEFILE_NULL)) { table.putToParameters( serdeConstants.SERIALIZATION_NULL_FORMAT, StringEscapeUtils.unescapeJava( tableDesc.getMeta().getOption(StorageConstants.SEQUENCEFILE_NULL))); table.getParameters().remove(StorageConstants.SEQUENCEFILE_NULL); } } else { if (tableDesc.getMeta().getStoreType().equalsIgnoreCase(BuiltinStorages.PARQUET)) { sd.setInputFormat(parquet.hive.DeprecatedParquetInputFormat.class.getName()); sd.setOutputFormat(parquet.hive.DeprecatedParquetOutputFormat.class.getName()); sd.getSerdeInfo() .setSerializationLib(parquet.hive.serde.ParquetHiveSerDe.class.getName()); } else { throw new UnsupportedException( tableDesc.getMeta().getStoreType() + " in HivecatalogStore"); } } sd.setSortCols(new ArrayList<Order>()); table.setSd(sd); client.getHiveClient().createTable(table); } catch (Throwable t) { throw new TajoInternalError(t); } finally { if (client != null) client.release(); } }