diff --git a/.gitignore b/.gitignore index 5bca77cf2c..276de0d2e6 100644 --- a/.gitignore +++ b/.gitignore @@ -17,3 +17,6 @@ atlassian-ide-plugin.xml # Patch files *.patch + +# Antlr files +*.tokens diff --git a/.travis.yml b/.travis.yml index 61f816c2ee..8627a9201c 100644 --- a/.travis.yml +++ b/.travis.yml @@ -32,6 +32,10 @@ matrix: - jdk: oraclejdk8 env: PATH=$PATH:$HOME/local/bin HADOOP_FLAG="" TEST_FLAG="test -Ptest-storage-pgsql -DLOG_LEVEL=WARN -pl org.apache.tajo:tajo-storage-pgsql" + # Test tajo-storage-mysql + - jdk: oraclejdk8 + env: PATH=$PATH:$HOME/local/bin HADOOP_FLAG="" TEST_FLAG="test -Ptest-storage-mysql -DLOG_LEVEL=WARN -pl org.apache.tajo:tajo-storage-mysql" + # Test tajo-storage-hdfs for hadoop-2.7.1 - jdk: oraclejdk8 env: PATH=$PATH:$HOME/local/bin HADOOP_FLAG="-Dhadoop.version=2.7.1" TEST_FLAG="test -pl org.apache.tajo:tajo-storage-hdfs" diff --git a/CHANGES b/CHANGES index eec145c14e..f2bd9642ba 100644 --- a/CHANGES +++ b/CHANGES @@ -4,7 +4,14 @@ Release 0.12.0 - unreleased NEW FEATURES - TAJO-1686: Allow Tajo to use Hive UDF. (jihoon) + TAJO-2165: Add 'ALTER TABLE UNSET PROPERTY' statement to Tajo DDL. + (Lee Dongjin via jihoon) + + TAJO-2160: Implement string_agg function. (Byunghwa Yun via jinho) + + TAJO-2148: Implement an example HTTP tablespace. (jihoon) + + TAJO-1686: Allow Tajo to use Hive UDF. (Jongyoung Park via jihoon) TAJO-2122: PullServer as an Auxiliary service of Yarn. (jihoon) @@ -14,6 +21,10 @@ Release 0.12.0 - unreleased IMPROVEMENT + TAJO-2146: Fragment interface cleanup. (jihoon) + + TAJO-2129: Apply new type implementation to Schema and Catalog. (hyunsik) + TAJO-2071: Supporting DATE type in Parquet format. (Jongyoung Park via hyunsik) @@ -147,6 +158,22 @@ Release 0.12.0 - unreleased BUG FIXES + TAJO-2168: NullPointerException occurs when a simple query contains a python udf. + (jihoon) + + TAJO-2164: SequenceFile print wrong values with TextSerializerDeserializer. + (jaehwa) + + TAJO-2163: No error message is showed when query execution failed. + (Wonjun Hong via jihoon) + + TAJO-2150: Fix invalid implementation of date time in jdbc. (jinho) + + TAJO-2151: Fix broken CI. (jihoon) + + TAJO-2158: The concat_ws function can't support a tab separator. + (Byunghwa Yun via jihoon) + TAJO-2143: Fix race condition in task history writer. (jinho) TAJO-2140: TajoInternalError does not contains reason stack trace. (jinho) @@ -266,6 +293,13 @@ Release 0.12.0 - unreleased TASKS + TAJO-2154: Refactor Datum to use new Type implementation. (hyunsik) + + TAJO-2027: Writing Hive UDF integration document. (Jongyoung via hyunsik) + + TAJO-2157: EvalNodeSerializer/Deserializer should use new Type + implementation. (hyunsik) + TAJO-2145: Error codes based on errno.h need prefix. (Jongyoung Park via hyunsik) @@ -363,6 +397,11 @@ Release 0.12.0 - unreleased SUB TASKS + TAJO-2156: Create GeoIP functions taking various types instead of INET4 type. (jongyoung) + + TAJO-2130: Remove inet4 data type. + (Contributed by Jongyoung Park. Committed by jihoon) + TAJO-1997: Registering UDF, it needs to check duplication. (Contributed by Jongyoung Park. Committed by jihoon) diff --git a/pom.xml b/pom.xml index 71c062bbfa..32c33f423b 100644 --- a/pom.xml +++ b/pom.xml @@ -98,6 +98,7 @@ tajo-metrics tajo-core-tests tajo-cluster-tests + tajo-tablespace-example tajo-dist @@ -144,7 +145,7 @@ maven-assembly-plugin - 2.3 + 2.4.1 org.apache.maven.plugins diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTable.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTable.java index af6e902bbd..d65c87cbd8 100644 --- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTable.java +++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTable.java @@ -25,6 +25,7 @@ import org.apache.tajo.util.TUtil; import java.util.HashMap; +import java.util.List; import java.util.Map; public class AlterTable extends Expr { @@ -43,6 +44,8 @@ public class AlterTable extends Expr { private AlterTableOpType alterTableOpType; @Expose @SerializedName("TableProperties") private Map params; + @Expose @SerializedName("UnsetPropertyKeys") + private List unsetPropertyKeys; @Expose @SerializedName("Columns") ColumnReferenceExpr [] columns; @@ -137,6 +140,14 @@ public void setParams(Map params) { this.params = params; } + public List getUnsetPropertyKeys() { + return unsetPropertyKeys; + } + + public void setUnsetPropertyKeys(List unsetPropertyKeys) { + this.unsetPropertyKeys = unsetPropertyKeys; + } + public boolean isPurge() { return purge; } diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTableOpType.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTableOpType.java index 89daef033e..b9f39af976 100644 --- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTableOpType.java +++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/AlterTableOpType.java @@ -18,5 +18,5 @@ package org.apache.tajo.algebra; public enum AlterTableOpType { - RENAME_TABLE, RENAME_COLUMN, ADD_COLUMN, ADD_PARTITION, DROP_PARTITION, SET_PROPERTY, REPAIR_PARTITION + RENAME_TABLE, RENAME_COLUMN, ADD_COLUMN, ADD_PARTITION, DROP_PARTITION, SET_PROPERTY, UNSET_PROPERTY, REPAIR_PARTITION } diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnDefinition.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnDefinition.java index 2a829e1950..aa26bfc5ad 100644 --- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnDefinition.java +++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/ColumnDefinition.java @@ -24,11 +24,6 @@ public class ColumnDefinition extends DataTypeExpr { @Expose @SerializedName("ColumnDefName") String columnName; - public ColumnDefinition(String columnName, String dataType) { - super(dataType); - this.columnName = columnName; - } - public ColumnDefinition(String columnName, DataTypeExpr dataType) { super(dataType.getTypeName()); @@ -42,15 +37,17 @@ public ColumnDefinition(String columnName, DataTypeExpr dataType) { } } - // nested records if (dataType.isRecordType()) { this.recordType = dataType.recordType; } - // map type if (dataType.isMapType()) { this.mapType = dataType.mapType; } + + if (dataType.isArrayType()) { + this.arrayType = dataType.arrayType; + } } public String getColumnName() { diff --git a/tajo-algebra/src/main/java/org/apache/tajo/algebra/DataTypeExpr.java b/tajo-algebra/src/main/java/org/apache/tajo/algebra/DataTypeExpr.java index d63532dbb3..bd5caac91b 100644 --- a/tajo-algebra/src/main/java/org/apache/tajo/algebra/DataTypeExpr.java +++ b/tajo-algebra/src/main/java/org/apache/tajo/algebra/DataTypeExpr.java @@ -19,11 +19,15 @@ package org.apache.tajo.algebra; import com.google.common.base.Objects; +import com.google.common.base.Preconditions; import com.google.gson.annotations.Expose; import com.google.gson.annotations.SerializedName; +import org.apache.tajo.Assert; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.util.TUtil; +import static org.apache.tajo.Assert.assertNotNull; + public class DataTypeExpr extends Expr { @Expose @SerializedName("DataTypeName") String typeName; @@ -31,6 +35,8 @@ public class DataTypeExpr extends Expr { Integer lengthOrPrecision; @Expose @SerializedName("Scale") Integer scale; + @Expose @SerializedName("Array") + ArrayType arrayType; // not null if the type is ARRAY @Expose @SerializedName("Record") RecordType recordType; // not null if the type is RECORD @Expose @SerializedName("Map") @@ -38,18 +44,27 @@ public class DataTypeExpr extends Expr { public DataTypeExpr(String typeName) { super(OpType.DataType); + assertNotNull(typeName); this.typeName = typeName; } + public DataTypeExpr(ArrayType array) { + super(OpType.DataType); + assertNotNull(array); + this.typeName = Type.ARRAY.name(); + this.arrayType = array; + } + public DataTypeExpr(RecordType record) { super(OpType.DataType); + assertNotNull(record); this.typeName = Type.RECORD.name(); this.recordType = record; } public DataTypeExpr(MapType map) { super(OpType.DataType); - // RECORD = 51 in DataTypes.proto + assertNotNull(map); this.typeName = Type.MAP.name(); this.mapType = map; } @@ -59,15 +74,34 @@ public String getTypeName() { } public boolean isPrimitiveType() { - return !this.isRecordType() && !isMapType(); + return !isArrayType()&& !isRecordType() && !isMapType(); + } + + public boolean isArrayType() { + return arrayType != null; } public boolean isRecordType() { - return this.typeName.equals(Type.RECORD.name()); + return recordType != null; } public boolean isMapType() { - return this.typeName.equals(Type.MAP.name()); + return mapType != null; + } + + public DataTypeExpr getElementType() { + Preconditions.checkState(isArrayType()); + return arrayType.type; + } + + public DataTypeExpr getKeyType() { + Preconditions.checkState(isMapType()); + return mapType.keyType; + } + + public DataTypeExpr getValueType() { + Preconditions.checkState(isMapType()); + return mapType.valueType; } public ColumnDefinition [] getNestedRecordTypes() { @@ -125,6 +159,27 @@ public Object clone() throws CloneNotSupportedException { return dataType; } + public static class ArrayType implements JsonSerializable, Cloneable { + @Expose + @SerializedName("type") + DataTypeExpr type; + + public ArrayType(DataTypeExpr elementType) { + this.type = elementType; + } + + @Override + public String toJson() { + return JsonHelper.toJson(this); + } + + public Object clone() throws CloneNotSupportedException { + ArrayType newMap = (ArrayType) super.clone(); + newMap.type = type; + return newMap; + } + } + public static class RecordType implements JsonSerializable, Cloneable { @Expose @SerializedName("Schema") ColumnDefinition [] schema; // not null if the type is RECORD diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java index b0acbb12a4..2c4d41e72b 100644 --- a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java +++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/AbstractCatalogClient.java @@ -32,6 +32,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListResponse; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.util.ProtoUtil; import java.io.Closeable; @@ -281,7 +282,7 @@ public final TableDesc getTableDesc(final String databaseName, final String tabl @Override public TableDesc getTableDesc(String qualifiedName) throws UndefinedTableException { - String[] splitted = CatalogUtil.splitFQTableName(qualifiedName); + String[] splitted = IdentifierUtil.splitFQTableName(qualifiedName); return getTableDesc(splitted[0], splitted[1]); } @@ -596,7 +597,7 @@ public final void createTable(final TableDesc desc) public void dropTable(String tableName) throws UndefinedDatabaseException, UndefinedTableException, InsufficientPrivilegeException { - String[] splitted = CatalogUtil.splitFQTableName(tableName); + String[] splitted = IdentifierUtil.splitFQTableName(tableName); final String databaseName = splitted[0]; final String simpleName = splitted[1]; @@ -617,7 +618,7 @@ public void dropTable(String tableName) @Override public final boolean existsTable(final String databaseName, final String tableName) { - if (CatalogUtil.isFQTableName(tableName)) { + if (IdentifierUtil.isFQTableName(tableName)) { throw new IllegalArgumentException( "tableName cannot be composed of multiple parts, but it is \"" + tableName + "\""); } @@ -640,7 +641,7 @@ public final boolean existsTable(final String databaseName, final String tableNa @Override public final boolean existsTable(final String tableName) { - String[] splitted = CatalogUtil.splitFQTableName(tableName); + String[] splitted = IdentifierUtil.splitFQTableName(tableName); return existsTable(splitted[0], splitted[1]); } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java index 9cbdbe0391..4b99d5646c 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/DataTypeUtil.java @@ -70,8 +70,6 @@ private static void putAcceptableType(TajoDataTypes.Type given, TajoDataTypes.Ty putAcceptableType(DATE, DATE); putAcceptableType(TEXT, TEXT); - - putAcceptableType(INET4, INET4); } public static boolean isUpperCastable(TajoDataTypes.Type define, TajoDataTypes.Type given) { @@ -87,8 +85,8 @@ public static boolean isUpperCastable(TajoDataTypes.Type define, TajoDataTypes.T */ public static org.apache.tajo.type.Type determineType(org.apache.tajo.type.Type left, org.apache.tajo.type.Type right) { - TajoDataTypes.Type rhsBaseType = right.baseType(); - switch (left.baseType()) { + TajoDataTypes.Type rhsBaseType = right.kind(); + switch (left.kind()) { case INT1: case INT2: diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java index a9dccbc536..b25dac9284 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableDesc.java @@ -17,6 +17,7 @@ */ package org.apache.tajo.catalog; +import com.google.common.collect.Sets; import com.google.gson.Gson; import com.google.gson.GsonBuilder; import com.google.gson.annotations.Expose; @@ -27,8 +28,10 @@ import org.apache.tajo.common.ProtoObject; import org.apache.tajo.json.GsonObject; import org.apache.tajo.util.KeyValueSet; +import org.apache.tajo.util.ProtoUtil; import java.net.URI; +import java.util.Set; import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTableDescProto; @@ -51,6 +54,8 @@ public class AlterTableDesc implements ProtoObject, GsonObj @Expose protected KeyValueSet properties; @Expose + protected Set unsetPropertyKeys; + @Expose protected URI newTablePath; // optional public AlterTableDesc() { @@ -59,6 +64,7 @@ public AlterTableDesc() { private void init() { this.properties = new KeyValueSet(); + this.unsetPropertyKeys = Sets.newHashSet(); } public String getTableName() { @@ -125,6 +131,14 @@ public void setProperty(String key, String value) { this.properties.set(key, value); } + public Set getUnsetPropertyKeys() { + return unsetPropertyKeys; + } + + public void setUnsetPropertyKey(Set unsetPropertyKeys) { + this.unsetPropertyKeys = unsetPropertyKeys; + } + public String getProperty(String key) { return this.properties.get(key); } @@ -154,6 +168,7 @@ public AlterTableDesc clone() throws CloneNotSupportedException { newAlter.addColumn = addColumn; newAlter.partitionDesc = partitionDesc; newAlter.properties = (KeyValueSet)properties.clone(); + newAlter.unsetPropertyKeys = Sets.newHashSet(unsetPropertyKeys); newAlter.newTablePath = URI.create(newTablePath.toString()); return newAlter; } @@ -185,6 +200,9 @@ public AlterTableDescProto getProto() { if (null != this.properties) { builder.setParams(properties.getProto()); } + if (null != this.unsetPropertyKeys) { + builder.setUnsetPropertyKeys(ProtoUtil.convertStrings(unsetPropertyKeys)); + } switch (alterTableType) { case RENAME_TABLE: @@ -199,6 +217,9 @@ public AlterTableDescProto getProto() { case SET_PROPERTY: builder.setAlterTableType(CatalogProtos.AlterTableType.SET_PROPERTY); break; + case UNSET_PROPERTY: + builder.setAlterTableType(CatalogProtos.AlterTableType.UNSET_PROPERTY); + break; case ADD_PARTITION: builder.setAlterTableType(CatalogProtos.AlterTableType.ADD_PARTITION); break; diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableType.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableType.java index a562d9f034..f395763693 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableType.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/AlterTableType.java @@ -18,5 +18,5 @@ package org.apache.tajo.catalog; public enum AlterTableType { - RENAME_TABLE, RENAME_COLUMN, ADD_COLUMN, ADD_PARTITION, DROP_PARTITION, SET_PROPERTY + RENAME_TABLE, RENAME_COLUMN, ADD_COLUMN, ADD_PARTITION, DROP_PARTITION, SET_PROPERTY, UNSET_PROPERTY } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java index f2acf983db..6acbc34e7f 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogConstants.java @@ -19,10 +19,6 @@ package org.apache.tajo.catalog; public class CatalogConstants { - public final static String IDENTIFIER_DELIMITER_REGEXP = "\\."; - public final static String IDENTIFIER_DELIMITER = "."; - public final static String IDENTIFIER_QUOTE_STRING = "\""; - public final static int MAX_IDENTIFIER_LENGTH = 128; // Linux and BSD's max username length is 32. For compatibility with other systems, we should follow it. public final static int MAX_USERNAME_LENGTH = 32; public final static int MAX_STATEMENT_LENGTH = 128 * 1024; diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java index 2b20907c44..892e5c0f38 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java @@ -18,23 +18,26 @@ package org.apache.tajo.catalog; -import com.google.common.base.Preconditions; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.apache.hadoop.fs.Path; import org.apache.tajo.BuiltinStorages; import org.apache.tajo.DataTypeUtil; -import org.apache.tajo.TajoConstants; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.partition.PartitionDesc; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.proto.CatalogProtos.*; +import org.apache.tajo.catalog.proto.CatalogProtos.PartitionKeyProto; +import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto; +import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto; +import org.apache.tajo.catalog.proto.CatalogProtos.TableIdentifierProto; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.exception.InvalidOperationException; import org.apache.tajo.exception.UndefinedOperatorException; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.Pair; @@ -51,271 +54,14 @@ public class CatalogUtil { - /** - * Normalize an identifier. Normalization means a translation from a identifier to be a refined identifier name. - * - * Identifier can be composed of multiple parts as follows: - *
-   *   database_name.table_name.column_name
-   * 
- * - * Each regular identifier part can be composed alphabet ([a-z][A-Z]), number([0-9]), and underscore([_]). - * Also, the first letter must be an alphabet character. - * - * normalizeIdentifier normalizes each part of an identifier. - * - * In detail, for each part, it performs as follows: - *
    - *
  • changing a part without double quotation to be lower case letters
  • - *
  • eliminating double quotation marks from identifier
  • - *
- * - * @param identifier The identifier to be normalized - * @return The normalized identifier - */ - public static String normalizeIdentifier(String identifier) { - if (identifier == null || identifier.equals("")) { - return identifier; - } - String [] splitted = identifier.split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP); - - StringBuilder sb = new StringBuilder(); - boolean first = true; - for (String part : splitted) { - if (first) { - first = false; - } else { - sb.append(CatalogConstants.IDENTIFIER_DELIMITER); - } - sb.append(normalizeIdentifierPart(part)); - } - return sb.toString(); - } - - public static String normalizeIdentifierPart(String part) { - return isDelimited(part) ? stripQuote(part) : part.toLowerCase(); - } - - /** - * Denormalize an identifier. Denormalize means a translation from a stored identifier - * to be a printable identifier name. - * - * In detail, for each part, it performs as follows: - *
    - *
  • changing a part including upper case character or non-ascii character to be lower case letters
  • - *
  • eliminating double quotation marks from identifier
  • - *
- * - * @param identifier The identifier to be normalized - * @return The denormalized identifier - */ - public static String denormalizeIdentifier(String identifier) { - String [] splitted = identifier.split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP); - - StringBuilder sb = new StringBuilder(); - boolean first = true; - for (String part : splitted) { - if (first) { - first = false; - } else { - sb.append(CatalogConstants.IDENTIFIER_DELIMITER); - } - sb.append(denormalizePart(part)); - } - return sb.toString(); - } - - public static String denormalizePart(String identifier) { - if (isShouldBeQuoted(identifier)) { - return StringUtils.doubleQuote(identifier); - } else { - return identifier; - } - } - - public static boolean isShouldBeQuoted(String columnName) { - for (char character : columnName.toCharArray()) { - if (Character.isUpperCase(character)) { - return true; - } - - if (!StringUtils.isPartOfAnsiSQLIdentifier(character)) { - return true; - } - - if (RESERVED_KEYWORDS_SET.contains(columnName.toUpperCase())) { - return true; - } - } - - return false; - } - - public static String stripQuote(String str) { - return str.substring(1, str.length() - 1); - } - - public static boolean isDelimited(String identifier) { - boolean openQuote = identifier.charAt(0) == '"'; - boolean closeQuote = identifier.charAt(identifier.length() - 1) == '"'; - - // if at least one quote mark exists, the identifier must be grater than equal to 2 characters, - if (openQuote ^ closeQuote && identifier.length() < 2) { - throw new IllegalArgumentException("Invalid Identifier: " + identifier); - } - - // does not allow the empty identifier (''), - if (openQuote && closeQuote && identifier.length() == 2) { - throw new IllegalArgumentException("zero-length delimited identifier: " + identifier); - } - - // Ensure the quote open and close - return openQuote && closeQuote; - } - - /** - * True if a given name is a simple identifier, meaning is not a dot-chained name. - * - * @param columnOrTableName Column or Table name to be checked - * @return True if a given name is a simple identifier. Otherwise, it will return False. - */ - public static boolean isSimpleIdentifier(String columnOrTableName) { - return columnOrTableName.split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP).length == 1; - } - - public static boolean isFQColumnName(String tableName) { - return tableName.split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP).length == 3; - } - - public static boolean isFQTableName(String tableName) { - int lastDelimiterIdx = tableName.lastIndexOf(CatalogConstants.IDENTIFIER_DELIMITER); - return lastDelimiterIdx > -1; - } - - public static String [] splitFQTableName(String qualifiedName) { - String [] splitted = CatalogUtil.splitTableName(qualifiedName); - if (splitted.length == 1) { - throw new IllegalArgumentException("Table name is expected to be qualified, but was \"" - + qualifiedName + "\"."); - } - return splitted; - } - - public static String [] splitTableName(String tableName) { - int lastDelimiterIdx = tableName.lastIndexOf(CatalogConstants.IDENTIFIER_DELIMITER); - if (lastDelimiterIdx > -1) { - return new String [] { - tableName.substring(0, lastDelimiterIdx), - tableName.substring(lastDelimiterIdx + 1, tableName.length()) - }; - } else { - return new String [] {tableName}; - } - } - - public static String buildFQName(String... identifiers) { - boolean first = true; - StringBuilder sb = new StringBuilder(); - for(String id : identifiers) { - if (first) { - first = false; - } else { - sb.append(CatalogConstants.IDENTIFIER_DELIMITER); - } - - sb.append(id); - } - - return sb.toString(); - } - - public static Pair separateQualifierAndName(String name) { - Preconditions.checkArgument(isFQTableName(name), "Must be a qualified name."); - return new Pair<>(extractQualifier(name), extractSimpleName(name)); - } - - /** - * Extract a qualification name from an identifier. - * - * For example, consider a table identifier like 'database1.table1'. - * In this case, this method extracts 'database1'. - * - * @param name The identifier to be extracted - * @return The extracted qualifier - */ - public static String extractQualifier(String name) { - int lastDelimiterIdx = name.lastIndexOf(CatalogConstants.IDENTIFIER_DELIMITER); - if (lastDelimiterIdx > -1) { - return name.substring(0, lastDelimiterIdx); - } else { - return TajoConstants.EMPTY_STRING; - } - } - - /** - * Extract a simple name from an identifier. - * - * For example, consider a table identifier like 'database1.table1'. - * In this case, this method extracts 'table1'. - * - * @param name The identifier to be extracted - * @return The extracted simple name - */ - public static String extractSimpleName(String name) { - int lastDelimiterIdx = name.lastIndexOf(CatalogConstants.IDENTIFIER_DELIMITER); - if (lastDelimiterIdx > -1) { - // plus one means skipping a delimiter. - return name.substring(lastDelimiterIdx + 1, name.length()); - } else { - return name; - } - } - - public static String getCanonicalTableName(String databaseName, String tableName) { - StringBuilder sb = new StringBuilder(databaseName); - sb.append(CatalogConstants.IDENTIFIER_DELIMITER); - sb.append(tableName); - return sb.toString(); - } - - public static String getBackwardCompitableDataFormat(String dataFormat) { - return getDataFormatAsString(asDataFormat(dataFormat)); - } - - public static String getDataFormatAsString(final DataFormat type) { - if (type == DataFormat.TEXTFILE) { - return BuiltinStorages.TEXT; - } else { - return type.name(); - } - } - - public static DataFormat asDataFormat(final String typeStr) { - if (typeStr.equalsIgnoreCase("CSV")) { - return DataFormat.TEXTFILE; - } else if (typeStr.equalsIgnoreCase(DataFormat.RAW.name())) { - return CatalogProtos.DataFormat.RAW; - } else if (typeStr.equalsIgnoreCase(CatalogProtos.DataFormat.ROWFILE.name())) { - return DataFormat.ROWFILE; - } else if (typeStr.equalsIgnoreCase(DataFormat.RCFILE.name())) { - return DataFormat.RCFILE; - } else if (typeStr.equalsIgnoreCase(CatalogProtos.DataFormat.ORC.name())) { - return CatalogProtos.DataFormat.ORC; - } else if (typeStr.equalsIgnoreCase(DataFormat.PARQUET.name())) { - return DataFormat.PARQUET; - } else if (typeStr.equalsIgnoreCase(DataFormat.SEQUENCEFILE.name())) { - return DataFormat.SEQUENCEFILE; - } else if (typeStr.equalsIgnoreCase(DataFormat.AVRO.name())) { - return CatalogProtos.DataFormat.AVRO; - } else if (typeStr.equalsIgnoreCase(BuiltinStorages.TEXT)) { - return CatalogProtos.DataFormat.TEXTFILE; - } else if (typeStr.equalsIgnoreCase(CatalogProtos.DataFormat.JSON.name())) { - return CatalogProtos.DataFormat.JSON; - } else if (typeStr.equalsIgnoreCase(CatalogProtos.DataFormat.HBASE.name())) { - return CatalogProtos.DataFormat.HBASE; - } else { - return null; + final String upperDataFormat = dataFormat.toUpperCase(); + switch (upperDataFormat) { + case "CSV": + case "TEXTFILE": + return BuiltinStorages.TEXT; + default: + return dataFormat; } } @@ -400,7 +146,7 @@ public static DataType newDataTypeWithLen(Type type, int length) { } public static String columnToDDLString(Column column) { - StringBuilder sb = new StringBuilder(denormalizeIdentifier(column.getSimpleName())); + StringBuilder sb = new StringBuilder(IdentifierUtil.denormalizeIdentifier(column.getSimpleName())); sb.append(" ").append(column.getDataType().getType()); if (column.getDataType().hasLength()) { sb.append(" (").append(column.getDataType().getLength()).append(")"); @@ -738,32 +484,8 @@ public static void closeQuietly(Statement stmt, ResultSet res) { } } - public static final Set RESERVED_KEYWORDS_SET = new HashSet<>(); - - static final String [] RESERVED_KEYWORDS = { - "AS", "ALL", "AND", "ANY", "ASYMMETRIC", "ASC", - "BOTH", - "CASE", "CAST", "CREATE", "CROSS", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", - "DESC", "DISTINCT", - "END", "ELSE", "EXCEPT", - "FALSE", "FULL", "FROM", - "GROUP", - "HAVING", - "ILIKE", "IN", "INNER", "INTERSECT", "INTO", "IS", - "JOIN", - "LEADING", "LEFT", "LIKE", "LIMIT", - "NATURAL", "NOT", "NULL", - "ON", "OUTER", "OR", "ORDER", - "RIGHT", - "SELECT", "SOME", "SYMMETRIC", - "TABLE", "THEN", "TRAILING", "TRUE", - "OVER", - "UNION", "UNIQUE", "USING", - "WHEN", "WHERE", "WINDOW", "WITH" - }; - static { - Collections.addAll(RESERVED_KEYWORDS_SET, RESERVED_KEYWORDS); + Collections.addAll(IdentifierUtil.RESERVED_KEYWORDS_SET, IdentifierUtil.RESERVED_KEYWORDS); } public static AlterTableDesc renameColumn(String tableName, String oldColumName, String newColumName, @@ -804,6 +526,14 @@ public static AlterTableDesc setProperty(String tableName, KeyValueSet params, A return alterTableDesc; } + public static AlterTableDesc unsetProperty(String tableName, String[] propertyKeys) { + final AlterTableDesc alterTableDesc = new AlterTableDesc(); + alterTableDesc.setTableName(tableName); + alterTableDesc.setUnsetPropertyKey(Sets.newHashSet(propertyKeys)); + alterTableDesc.setAlterTableType(AlterTableType.UNSET_PROPERTY); + return alterTableDesc; + } + /** * Converts passed parameters to a AlterTableDesc. This method would be called when adding a partition or dropping * a table. This creates AlterTableDesc that is a wrapper class for protocol buffer. @@ -991,8 +721,6 @@ public enum Direction { TUtil.putToNestedMap(OPERATION_CASTING_MAP, Type.INTERVAL, Type.DATE, Type.TIMESTAMP); TUtil.putToNestedMap(CASTING_DIRECTION_MAP, Type.INTERVAL, Type.DATE, Direction.RHS); - - TUtil.putToNestedMap(OPERATION_CASTING_MAP, Type.INET4, Type.INET4, Type.INET4); } // table default properties @@ -1022,7 +750,7 @@ public static KeyValueSet newDefaultProperty(String dataFormat, TajoConf conf) { options.set(StorageConstants.RCFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE); } else if (dataFormat.equalsIgnoreCase("SEQUENCEFILE")) { options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE); - options.set(StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); + options.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); } else if (dataFormat.equalsIgnoreCase("PARQUET")) { options.set(BLOCK_SIZE, StorageConstants.PARQUET_DEFAULT_BLOCK_SIZE); options.set(PAGE_SIZE, StorageConstants.PARQUET_DEFAULT_PAGE_SIZE); @@ -1045,7 +773,7 @@ public static KeyValueSet newDefaultProperty(String dataFormat, TajoConf conf) { public static String getUnifiedSimpleColumnName(Schema originalSchema, String[] columnNames) { String[] simpleNames = new String[columnNames.length]; for (int i = 0; i < simpleNames.length; i++) { - String[] identifiers = columnNames[i].split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP); + String[] identifiers = columnNames[i].split(IdentifierUtil.IDENTIFIER_DELIMITER_REGEXP); simpleNames[i] = identifiers[identifiers.length-1]; } Arrays.sort(simpleNames, new ColumnPosComparator(originalSchema)); diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java index 641b674cc8..ddf0c8329c 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java @@ -26,24 +26,27 @@ import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.json.GsonObject; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.type.Type; +import org.apache.tajo.type.TypeProtobufEncoder; /** * Describes a column. It is an immutable object. */ +@Deprecated public class Column implements ProtoObject, GsonObject { @Expose protected String name; - @Expose protected TypeDesc typeDesc; + @Expose protected Type type; /** * Column Constructor * * @param name field name - * @param typeDesc Type description + * @param type Type description */ - public Column(String name, TypeDesc typeDesc) { + public Column(String name, TypeDesc type) { this.name = name; - this.typeDesc = typeDesc; + this.type = TypeConverter.convert(type); } /** @@ -70,22 +73,13 @@ public Column(String name, TajoDataTypes.Type type) { * @param type Type */ public Column(String name, Type type) { - this(name, TypeConverter.convert(type)); - } - - /** - * - * @param name Column name - * @param type Data Type - * @param typeLength The length of type - */ - public Column(String name, TajoDataTypes.Type type, int typeLength) { - this(name, CatalogUtil.newDataTypeWithLen(type, typeLength)); + this.name = name; + this.type = type; } public Column(ColumnProto proto) { name = proto.getName(); - typeDesc = new TypeDesc(proto.getDataType()); + type = TypeProtobufEncoder.decode(proto.getType()); } /** @@ -93,7 +87,7 @@ public Column(ColumnProto proto) { * @return True if a column includes a table name. Otherwise, it returns False. */ public boolean hasQualifier() { - return name.split(CatalogConstants.IDENTIFIER_DELIMITER_REGEXP).length > 1; + return name.split(IdentifierUtil.IDENTIFIER_DELIMITER_REGEXP).length > 1; } /** @@ -109,7 +103,7 @@ public String getQualifiedName() { * @return The qualifier */ public String getQualifier() { - return CatalogUtil.extractQualifier(name); + return IdentifierUtil.extractQualifier(name); } /** @@ -117,7 +111,7 @@ public String getQualifier() { * @return The simple name without qualifications */ public String getSimpleName() { - return CatalogUtil.extractSimpleName(name); + return IdentifierUtil.extractSimpleName(name); } /** @@ -126,7 +120,14 @@ public String getSimpleName() { * @return TypeDesc */ public TypeDesc getTypeDesc() { - return this.typeDesc; + return TypeConverter.convert(this.type); + } + + /** + * @return Type which includes domain type and scale. + */ + public Type getType() { + return this.type; } /** @@ -134,21 +135,20 @@ public TypeDesc getTypeDesc() { * @return DataType which includes domain type and scale. */ public DataType getDataType() { - return this.typeDesc.dataType; + return TypeConverter.convert(this.type).getDataType(); } @Override public boolean equals(Object o) { if (o instanceof Column) { Column another = (Column)o; - return name.equals(another.name) && typeDesc.equals(another.typeDesc); + return name.equals(another.name) && type.equals(another.type); } return false; } public int hashCode() { - return Objects.hashCode(name, typeDesc); - + return Objects.hashCode(name, type); } /** @@ -160,13 +160,13 @@ public ColumnProto getProto() { ColumnProto.Builder builder = ColumnProto.newBuilder(); builder .setName(this.name) - .setDataType(this.typeDesc.getDataType()); + .setType(this.type.getProto()); return builder.build(); } public String toString() { StringBuilder sb = new StringBuilder(getQualifiedName()); - sb.append(" (").append(typeDesc.toString()).append(")"); + sb.append(" (").append(type.toString()).append(")"); return sb.toString(); } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java index e2250eb9ef..387c669b5e 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java @@ -21,6 +21,7 @@ import com.google.common.base.Function; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.catalog.proto.CatalogProtos.PartitionDescProto; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.StringUtils; @@ -36,11 +37,11 @@ public static String buildDDLForExternalTable(TableDesc desc) { StringBuilder sb = new StringBuilder(); sb.append("--\n") - .append("-- Name: ").append(CatalogUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;") + .append("-- Name: ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;") .append(" Storage: ").append(desc.getMeta().getDataFormat()); sb.append("\n-- Path: ").append(desc.getUri()); sb.append("\n--\n"); - sb.append("CREATE EXTERNAL TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName())); + sb.append("CREATE EXTERNAL TABLE ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())); buildSchema(sb, desc.getSchema()); buildUsingClause(sb, desc.getMeta()); buildWithClause(sb, desc.getMeta()); @@ -59,10 +60,10 @@ public static String buildDDLForBaseTable(TableDesc desc) { StringBuilder sb = new StringBuilder(); sb.append("--\n") - .append("-- Name: ").append(CatalogUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;") + .append("-- Name: ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;") .append(" Storage: ").append(desc.getMeta().getDataFormat()); sb.append("\n--\n"); - sb.append("CREATE TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName())); + sb.append("CREATE TABLE ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())); buildSchema(sb, desc.getSchema()); buildUsingClause(sb, desc.getMeta()); buildWithClause(sb, desc.getMeta()); @@ -79,11 +80,11 @@ public static String buildDDLForIndex(IndexDesc desc) { StringBuilder sb = new StringBuilder(); sb.append("--\n") - .append("-- Name: ").append(CatalogUtil.denormalizeIdentifier(desc.getName())).append("; Type: INDEX;") + .append("-- Name: ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())).append("; Type: INDEX;") .append(" Index Method: ").append(desc.getIndexMethod()); sb.append("\n--\n"); - sb.append("CREATE INDEX ").append(CatalogUtil.denormalizeIdentifier(desc.getName())); - sb.append(" on ").append(CatalogUtil.denormalizeIdentifier(desc.getTableName())).append(" ( "); + sb.append("CREATE INDEX ").append(IdentifierUtil.denormalizeIdentifier(desc.getName())); + sb.append(" on ").append(IdentifierUtil.denormalizeIdentifier(desc.getTableName())).append(" ( "); for (SortSpec sortSpec : desc.getKeySortSpecs()) { sb.append(sortSpec.getSortKey().getQualifiedName()).append(" "); @@ -108,7 +109,7 @@ public static void buildSchema(StringBuilder sb, Schema schema) { sb.append(", "); } - sb.append(CatalogUtil.denormalizeIdentifier(column.getSimpleName())).append(" "); + sb.append(IdentifierUtil.denormalizeIdentifier(column.getSimpleName())).append(" "); TypeDesc typeDesc = column.getTypeDesc(); sb.append(typeDesc); } @@ -177,7 +178,7 @@ private static void buildPartitionClause(StringBuilder sb, TableDesc desc) { */ public static String buildDDLForAddPartition(TableDesc table, PartitionDescProto partition) { StringBuilder sb = new StringBuilder(); - sb.append("ALTER TABLE ").append(CatalogUtil.denormalizeIdentifier(table.getName())) + sb.append("ALTER TABLE ").append(IdentifierUtil.denormalizeIdentifier(table.getName())) .append(" ADD IF NOT EXISTS PARTITION ("); diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FieldConverter.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FieldConverter.java index 616b44e5bd..1b34429b78 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FieldConverter.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FieldConverter.java @@ -21,16 +21,12 @@ import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.exception.NotImplementedException; import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.schema.Field; import org.apache.tajo.schema.Identifier; -import org.apache.tajo.schema.IdentifierPolicy; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.schema.QualifiedIdentifier; -import org.apache.tajo.schema.Schema; -import org.apache.tajo.schema.Schema.NamedPrimitiveType; -import org.apache.tajo.schema.Schema.NamedStructType; -import org.apache.tajo.type.*; import javax.annotation.Nullable; import java.util.Collection; @@ -42,58 +38,20 @@ public static QualifiedIdentifier toQualifiedIdentifier(String name) { final Collection identifiers = Collections2.transform(elems, new Function() { @Override public Identifier apply(@Nullable String input) { - boolean needQuote = CatalogUtil.isShouldBeQuoted(input); - return Identifier._(input, needQuote); + return Identifier._(input, IdentifierUtil.isShouldBeQuoted(input)); } }); return QualifiedIdentifier.$(identifiers); } - public static TypeDesc convert(Schema.NamedType src) { - if (src instanceof NamedStructType) { - NamedStructType structType = (NamedStructType) src; - - ImmutableList.Builder fields = ImmutableList.builder(); - for (Schema.NamedType t: structType.fields()) { - fields.add(new Column(t.name().raw(IdentifierPolicy.DefaultPolicy()), convert(t))); - } - - return new TypeDesc(SchemaBuilder.builder().addAll(fields.build()).build()); - } else { - final NamedPrimitiveType namedType = (NamedPrimitiveType) src; - final Type type = namedType.type(); - if (type instanceof Char) { - Char charType = (Char) type; - return new TypeDesc(CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.CHAR, charType.length())); - } else if (type instanceof Varchar) { - Varchar varcharType = (Varchar) type; - return new TypeDesc(CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.VARCHAR, varcharType.length())); - } else if (type instanceof Numeric) { - Numeric numericType = (Numeric) type; - return new TypeDesc(CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.NUMERIC, numericType.precision())); - } else if (type instanceof Protobuf) { - Protobuf protobuf = (Protobuf) type; - return new TypeDesc(CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, protobuf.getMessageName())); - } else { - return new TypeDesc(TypeConverter.convert(namedType.type())); - } + public static Field convert(Column column) { + if (column.type.isStruct() && column.getTypeDesc().getNestedSchema() == null) { + throw new TajoRuntimeException(new NotImplementedException("record type projection")); } + return new Field(toQualifiedIdentifier(column.getQualifiedName()), column.type); } - public static Schema.NamedType convert(Column column) { - if (column.getTypeDesc().getDataType().getType() == TajoDataTypes.Type.RECORD) { - - if (column.getTypeDesc().getNestedSchema() == null) { - throw new TajoRuntimeException(new NotImplementedException("record type projection")); - } - - return new NamedStructType(toQualifiedIdentifier(column.getQualifiedName()), - TypeConverter.convert(column.getTypeDesc())); - - } else { - return new NamedPrimitiveType(toQualifiedIdentifier(column.getQualifiedName()), - TypeConverter.convert(column.getDataType()) - ); - } + public static Column convert(Field field) { + return new Column(field.name().interned(), field.type()); } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/ListSchemaBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/ListSchemaBuilder.java index e0c9d9e6b8..8f77489145 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/ListSchemaBuilder.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/ListSchemaBuilder.java @@ -20,30 +20,30 @@ import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableList; -import org.apache.tajo.schema.Schema.NamedType; +import org.apache.tajo.schema.Field; import java.util.Iterator; public class ListSchemaBuilder implements SchemaBuilder.SchemaCollector { - private final ImmutableList.Builder fields = new ImmutableList.Builder(); + private final ImmutableList.Builder fields = new ImmutableList.Builder(); @Override - public void add(NamedType namedType) { - fields.add(namedType); + public void add(Field field) { + fields.add(field); } @Override - public void addAll(Iterator fields) { + public void addAll(Iterator fields) { this.fields.addAll(fields); } @Override - public void addAll(Iterable fields) { + public void addAll(Iterable fields) { this.fields.addAll(fields); } @Override - public ImmutableCollection build() { + public ImmutableCollection build() { return fields.build(); } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/NestedPathUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/NestedPathUtil.java index bf874b95b0..afe7e3112d 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/NestedPathUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/NestedPathUtil.java @@ -73,7 +73,7 @@ private static Column lookupColumnInternal(Column currentColumn, String [] paths Column found = null; if (currentColumn.getDataType().getType() == Type.RECORD) { - found = currentColumn.typeDesc.nestedRecordSchema.getColumn(paths[depth]); + found = TypeConverter.convert(currentColumn.type).nestedRecordSchema.getColumn(paths[depth]); } if (found != null) { diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaBuilder.java index 35e38b53f7..582b5ee300 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaBuilder.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaBuilder.java @@ -22,18 +22,14 @@ import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableList; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.schema.Field; import org.apache.tajo.schema.QualifiedIdentifier; -import org.apache.tajo.schema.Schema.NamedPrimitiveType; -import org.apache.tajo.schema.Schema.NamedStructType; -import org.apache.tajo.schema.Schema.NamedType; import org.apache.tajo.type.Type; import javax.annotation.Nullable; -import java.util.Collection; import java.util.Iterator; import static org.apache.tajo.catalog.FieldConverter.toQualifiedIdentifier; -import static org.apache.tajo.schema.IdentifierPolicy.DefaultPolicy; /** * Builder for Schema @@ -42,10 +38,10 @@ public class SchemaBuilder { private final SchemaCollector fields; public interface SchemaCollector { - void add(NamedType field); - void addAll(Iterator fields); - void addAll(Iterable fields); - ImmutableCollection build(); + void add(Field field); + void addAll(Iterator fields); + void addAll(Iterable fields); + ImmutableCollection build(); } public static SchemaLegacy empty() { @@ -64,28 +60,19 @@ public static SchemaBuilder uniqueNameBuilder() { this.fields = collector; } - public SchemaBuilder add(NamedType namedType) { - fields.add(namedType); + public SchemaBuilder add(Field field) { + fields.add(field); return this; } public SchemaBuilder add(QualifiedIdentifier id, Type type) { - add(new NamedPrimitiveType(id, type)); - return this; - } - - public SchemaBuilder addStruct(QualifiedIdentifier id, Collection fields) { - add(new NamedStructType(id, fields)); + add(new Field(id, type)); return this; } @Deprecated public SchemaBuilder add(String name, TypeDesc legacyType) { - if (legacyType.getDataType().getType() == TajoDataTypes.Type.RECORD) { - addStruct(toQualifiedIdentifier(name), TypeConverter.convert(legacyType)); - } else { - add(toQualifiedIdentifier(name), TypeConverter.convert(legacyType.getDataType())); - } + add(toQualifiedIdentifier(name), TypeConverter.convert(legacyType)); return this; } @@ -109,9 +96,9 @@ public SchemaBuilder add(Column column) { @Deprecated public SchemaBuilder addAll(Iterable columns) { - return addAll2(columns, new Function() { + return addAll2(columns, new Function() { @Override - public NamedType apply(@Nullable Column input) { + public Field apply(@Nullable Column input) { return FieldConverter.convert(input); } }); @@ -119,9 +106,9 @@ public NamedType apply(@Nullable Column input) { @Deprecated public SchemaBuilder addAll(Column [] columns) { - return addAll2(columns, new Function() { + return addAll2(columns, new Function() { @Override - public NamedType apply(@Nullable Column input) { + public Field apply(@Nullable Column input) { return FieldConverter.convert(input); } }); @@ -152,21 +139,26 @@ public SchemaBuilder addAll(Iterator fields, Function fn) { return this; } - public SchemaBuilder addAll2(T [] fields, Function fn) { + public SchemaBuilder addAll2(Iterable fields) { + this.fields.addAll(fields); + return this; + } + + public SchemaBuilder addAll2(T [] fields, Function fn) { for (T t : fields) { add(fn.apply(t)); } return this; } - public SchemaBuilder addAll2(Iterable fields, Function fn) { + public SchemaBuilder addAll2(Iterable fields, Function fn) { for (T t : fields) { add(fn.apply(t)); } return this; } - public SchemaBuilder addAll2(Iterator fields, Function fn) { + public SchemaBuilder addAll2(Iterator fields, Function fn) { while(fields.hasNext()) { T t = fields.next(); add(fn.apply(t)); @@ -177,8 +169,8 @@ public SchemaBuilder addAll2(Iterator fields, Function fn) @Deprecated public SchemaLegacy build() { ImmutableList.Builder columns = new ImmutableList.Builder(); - for (NamedType namedType : fields.build()) { - columns.add(new Column(namedType.name().raw(DefaultPolicy()), FieldConverter.convert(namedType))); + for (Field field : fields.build()) { + columns.add(new Column(field.name().interned(), field.type())); } return new SchemaLegacy(columns.build()); diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaLegacy.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaLegacy.java index 2a16fa6e83..db1e566d59 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaLegacy.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaLegacy.java @@ -18,23 +18,27 @@ package org.apache.tajo.catalog; +import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.SchemaUtil.ColumnVisitor; import org.apache.tajo.catalog.json.CatalogGsonHelper; import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto; import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto; import org.apache.tajo.common.ProtoObject; -import org.apache.tajo.common.TajoDataTypes.DataType; -import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.exception.DuplicateColumnException; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.json.GsonObject; import org.apache.tajo.util.StringUtils; +import javax.annotation.Nullable; import java.util.*; +import static com.google.common.collect.Collections2.transform; + +@Deprecated public class SchemaLegacy implements Schema, ProtoObject, Cloneable, GsonObject { @Expose protected List fields = null; @@ -55,47 +59,18 @@ public SchemaLegacy() { public SchemaLegacy(SchemaProto proto) { init(); - List toBeAdded = new ArrayList<>(); - for (int i = 0; i < proto.getFieldsCount(); i++) { - deserializeColumn(toBeAdded, proto.getFieldsList(), i); - } + Collection toBeAdded = transform(proto.getFieldsList(), new Function() { + @Override + public Column apply(@Nullable ColumnProto proto) { + return new Column(proto); + } + }); for (Column c : toBeAdded) { addColumn(c); } } - /** - * This method transforms a list of ColumnProtos into a schema tree. - * It assumes that protos contains a list of ColumnProtos in the depth-first order. - * - * @param tobeAdded - * @param protos - * @param serializedColumnIndex - */ - private static void deserializeColumn(List tobeAdded, List protos, int serializedColumnIndex) { - ColumnProto columnProto = protos.get(serializedColumnIndex); - if (columnProto.getDataType().getType() == Type.RECORD) { - - // Get the number of child fields - int childNum = columnProto.getDataType().getNumNestedFields(); - // where is start index of nested fields? - int childStartIndex = tobeAdded.size() - childNum; - // Extract nested fields - List nestedColumns = new ArrayList<>(tobeAdded.subList(childStartIndex, childStartIndex + childNum)); - - // Remove nested fields from the the current level - for (int i = 0; i < childNum; i++) { - tobeAdded.remove(tobeAdded.size() - 1); - } - - // Add the nested fields to the list as a single record column - tobeAdded.add(new Column(columnProto.getName(), new TypeDesc(new SchemaLegacy(nestedColumns)))); - } else { - tobeAdded.add(new Column(protos.get(serializedColumnIndex))); - } - } - public SchemaLegacy(Schema schema) { new SchemaLegacy(schema.getRootColumns()); } @@ -139,7 +114,7 @@ public void setQualifier(String qualifier) { Column newColumn; for (Column c : columns) { - newColumn = new Column(qualifier + "." + c.getSimpleName(), c.typeDesc); + newColumn = new Column(qualifier + "." + c.getSimpleName(), c.type); addColumn(newColumn); } } @@ -200,7 +175,7 @@ public Column getColumn(String name) { Column columnPath = new Column( column.getQualifiedName() + NestedPathUtil.makePath(paths, 1), - actualColumn.typeDesc); + actualColumn.type); return columnPath; } else { @@ -424,13 +399,13 @@ public boolean containsAny(Collection columns) { return false; } - private SchemaLegacy addColumn(String name, TypeDesc typeDesc) { + private SchemaLegacy addColumn(String name, org.apache.tajo.type.Type type) { String normalized = name; if(fieldsByQualifiedName.containsKey(normalized)) { throw new TajoRuntimeException(new DuplicateColumnException(normalized)); } - Column newCol = new Column(normalized, typeDesc); + Column newCol = new Column(normalized, type); fields.add(newCol); fieldsByQualifiedName.put(newCol.getQualifiedName(), fields.size() - 1); List inputList = new ArrayList<>(); @@ -441,7 +416,7 @@ private SchemaLegacy addColumn(String name, TypeDesc typeDesc) { } private synchronized void addColumn(Column column) { - addColumn(column.getQualifiedName(), column.typeDesc); + addColumn(column.getQualifiedName(), column.type); } @Override @@ -472,34 +447,15 @@ public Object clone() throws CloneNotSupportedException { @Override public SchemaProto getProto() { SchemaProto.Builder builder = SchemaProto.newBuilder(); - SchemaProtoBuilder recursiveBuilder = new SchemaProtoBuilder(builder); - SchemaUtil.visitSchema(this, recursiveBuilder); + builder.addAllFields(Iterables.transform(getRootColumns(), new Function() { + @Override + public ColumnProto apply(@Nullable Column column) { + return column.getProto(); + } + })); return builder.build(); } - private static class SchemaProtoBuilder implements ColumnVisitor { - private SchemaProto.Builder builder; - public SchemaProtoBuilder(SchemaProto.Builder builder) { - this.builder = builder; - } - - @Override - public void visit(int depth, List path, Column column) { - - if (column.getDataType().getType() == Type.RECORD) { - DataType.Builder updatedType = DataType.newBuilder(column.getDataType()); - updatedType.setNumNestedFields(column.typeDesc.nestedRecordSchema.size()); - - ColumnProto.Builder updatedColumn = ColumnProto.newBuilder(column.getProto()); - updatedColumn.setDataType(updatedType); - - builder.addFields(updatedColumn.build()); - } else { - builder.addFields(column.getProto()); - } - } - } - @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java index c0b60a31bc..e77e7f9cb4 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SchemaUtil.java @@ -185,7 +185,7 @@ private static void visitInDepthFirstOrder(int depth, Column column) { if (column.getDataType().getType() == Type.RECORD) { - for (Column nestedColumn : column.typeDesc.nestedRecordSchema.getRootColumns()) { + for (Column nestedColumn : TypeConverter.convert(column.type).nestedRecordSchema.getRootColumns()) { List newPath = new ArrayList<>(path); newPath.add(column.getQualifiedName()); @@ -244,10 +244,6 @@ public static int getColByteSize(Column col) { return 4; case FLOAT8: return 8; - case INET4: - return 4; - case INET6: - return 16; case TEXT: return 256; case BLOB: diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SetSchemaBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SetSchemaBuilder.java index b66aac294e..4a4099ef56 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SetSchemaBuilder.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/SetSchemaBuilder.java @@ -20,47 +20,41 @@ import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableList; -import org.apache.commons.collections.collection.UnmodifiableCollection; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.schema.QualifiedIdentifier; -import org.apache.tajo.schema.Schema.NamedPrimitiveType; -import org.apache.tajo.schema.Schema.NamedStructType; -import org.apache.tajo.schema.Schema.NamedType; -import org.apache.tajo.type.Type; +import org.apache.tajo.schema.Field; -import java.util.*; - -import static org.apache.tajo.catalog.FieldConverter.toQualifiedIdentifier; -import static org.apache.tajo.schema.IdentifierPolicy.DefaultPolicy; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; public class SetSchemaBuilder implements SchemaBuilder.SchemaCollector { private final Set nameSet = new HashSet<>(); - private final ImmutableList.Builder fields = new ImmutableList.Builder(); + private final ImmutableList.Builder fields = new ImmutableList.Builder(); @Override - public void add(NamedType namedType) { - if (!nameSet.contains(namedType.name())) { - fields.add(namedType); - nameSet.add(namedType.name()); + public void add(Field field) { + if (!nameSet.contains(field.name())) { + fields.add(field); + nameSet.add(field.name()); } } @Override - public void addAll(Iterator fields) { + public void addAll(Iterator fields) { while (fields.hasNext()) { add(fields.next()); } } @Override - public void addAll(Iterable fields) { - for (NamedType n : fields) { + public void addAll(Iterable fields) { + for (Field n : fields) { add(n); } } @Override - public ImmutableCollection build() { + public ImmutableCollection build() { return fields.build(); } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java index f4164d4434..387cf006f5 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeConverter.java @@ -20,66 +20,25 @@ import com.google.common.collect.ImmutableList; import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.exception.NotImplementedException; -import org.apache.tajo.exception.TajoRuntimeException; -import org.apache.tajo.exception.UnsupportedException; -import org.apache.tajo.schema.Schema; +import org.apache.tajo.schema.Field; import org.apache.tajo.type.*; -import java.util.Collection; - +import static org.apache.tajo.catalog.CatalogUtil.newDataTypeWithLen; +import static org.apache.tajo.catalog.CatalogUtil.newSimpleDataType; +import static org.apache.tajo.common.TajoDataTypes.Type.*; import static org.apache.tajo.type.Type.*; public class TypeConverter { - public static Collection convert(TypeDesc type) { - ImmutableList.Builder fields = ImmutableList.builder(); - for (Column c : type.getNestedSchema().getRootColumns()) { - fields.add(FieldConverter.convert(c)); - } - return fields.build(); - } - - public static Type convert(TajoDataTypes.Type legacyBaseType) { - switch (legacyBaseType) { - case BOOLEAN: - return Bool; - case INT1: - return Int1; - case INT2: - return Int2; - case INT4: - return Int4; - case INT8: - return Int8; - case FLOAT4: - return Float4; - case FLOAT8: - return Float8; - case DATE: - return Date; - case TIME: - return Time; - case TIMESTAMP: - return Timestamp; - case INTERVAL: - return Interval; - case CHAR: - return Char(1); // default len = 1 - case TEXT: - return Text; - case BLOB: - return Blob; - case INET4: - return Inet4; - case RECORD: - throw new TajoRuntimeException(new NotImplementedException("record projection")); - case NULL_TYPE: - return Null; - case ANY: - return Any; - default: - throw new TajoRuntimeException(new UnsupportedException(legacyBaseType.name())); + public static Type convert(TypeDesc type) { + if (type.getDataType().getType() == TajoDataTypes.Type.RECORD) { + ImmutableList.Builder fields = ImmutableList.builder(); + for (Column c : type.getNestedSchema().getRootColumns()) { + fields.add(FieldConverter.convert(c)); + } + return Record(fields.build()); + } else { + return convert(type.dataType); } } @@ -96,26 +55,58 @@ public static Type convert(TajoDataTypes.DataType legacyType) { case PROTOBUF: return new Protobuf(legacyType.getCode()); default: - return convert(legacyType.getType()); + return TypeFactory.create(legacyType.getType()); } } - public static TajoDataTypes.DataType convert(Type type) { - switch (type.baseType()) { - case CHAR: - Char charType = (Char) type; - return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.CHAR, charType.length()); - case VARCHAR: - Varchar varcharType = (Varchar) type; - return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.VARCHAR, varcharType.length()); - case PROTOBUF: - Protobuf protobuf = (Protobuf) type; - return CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, protobuf.getMessageName()); - case NUMERIC: - Numeric numericType = (Numeric) type; - return CatalogUtil.newDataTypeWithLen(TajoDataTypes.Type.NUMERIC, numericType.precision()); + public static TypeDesc convert(Field src) { + return convert(src.type()); + } + + public static TypeDesc convert(Type type) { + switch (type.kind()) { + case CHAR: + Char charType = (Char) type; + return new TypeDesc(newDataTypeWithLen(TajoDataTypes.Type.CHAR, charType.length())); + case VARCHAR: + Varchar varcharType = (Varchar) type; + return new TypeDesc(newDataTypeWithLen(TajoDataTypes.Type.VARCHAR, varcharType.length())); + case NUMERIC: + Numeric numericType = (Numeric) type; + return new TypeDesc(newDataTypeWithLen(TajoDataTypes.Type.NUMERIC, numericType.precision())); + case PROTOBUF: + Protobuf protobuf = (Protobuf) type; + return new TypeDesc(CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, protobuf.getMessageName())); + case RECORD: + Record record = (Record) type; + ImmutableList.Builder fields = ImmutableList.builder(); + for (Field t: record.fields()) { + fields.add(new Column(t.name().interned(), convert(t))); + } + return new TypeDesc(SchemaBuilder.builder().addAll(fields.build()).build()); + + case ARRAY: + Array array = (Array) type; + Type elemType = array.elementType(); + switch (elemType.kind()) { + case INT1: + return new TypeDesc(newSimpleDataType(INT1_ARRAY)); + case INT2: + return new TypeDesc(newSimpleDataType(INT2_ARRAY)); + case INT4: + return new TypeDesc(newSimpleDataType(INT4_ARRAY)); + case INT8: + return new TypeDesc(newSimpleDataType(INT8_ARRAY)); + case FLOAT4: + return new TypeDesc(newSimpleDataType(FLOAT4_ARRAY)); + case FLOAT8: + return new TypeDesc(newSimpleDataType(FLOAT8_ARRAY)); default: - return CatalogUtil.newSimpleDataType(type.baseType()); + return new TypeDesc(newSimpleDataType(type.kind())); + } + + default: + return new TypeDesc(newSimpleDataType(type.kind())); } } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java index c145ecdd3c..27c2309043 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/json/CatalogGsonHelper.java @@ -48,6 +48,7 @@ private static Map> registerAdapters() { adapters.put(Datum.class, new DatumAdapter()); adapters.put(DataType.class, new DataTypeAdapter()); adapters.put(Schema.class, new SchemaAdapter()); + adapters.put(org.apache.tajo.type.Type.class, new TypeAdapter()); return adapters; } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java index c66500d2f3..47acb50f7c 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStats.java @@ -26,6 +26,7 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.common.TajoDataTypes.TypeProto; import org.apache.tajo.json.GsonObject; import org.apache.tajo.catalog.json.CatalogGsonHelper; import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto; @@ -35,6 +36,8 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.tajo.common.TajoDataTypes.Type.PROTOBUF; + public class TableStats implements ProtoObject, Cloneable, GsonObject { @Expose private Long numRows = null; // required @Expose private Long numBytes = null; // required @@ -85,13 +88,17 @@ public TableStats(CatalogProtos.TableStatsProto proto) { this.columnStatses = new ArrayList<>(); for (CatalogProtos.ColumnStatsProto colProto : proto.getColStatList()) { - if (colProto.getColumn().getDataType().getType() == TajoDataTypes.Type.PROTOBUF) { + if (peekType(colProto.getColumn().getType()) == PROTOBUF) { continue; } columnStatses.add(new ColumnStats(colProto)); } } + private static TajoDataTypes.Type peekType(TypeProto proto) { + return proto.getElements(proto.getElementsCount() - 1).getKind(); + } + public Long getNumRows() { return this.numRows; } diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto index d064c62586..f02bfbf227 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto +++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto @@ -25,22 +25,6 @@ option java_generate_equals_and_hash = true; import "DataTypes.proto"; import "PrimitiveProtos.proto"; -enum DataFormat { - MEM = 0; - TEXTFILE = 1; - RAW = 2; - RCFILE = 3; - ROWFILE = 4; - HCFILE = 5; - ORC = 6; - PARQUET = 7; - SEQUENCEFILE = 8; - AVRO = 9; - JSON = 10; - HBASE = 11; - SYSTEM = 12; -} - enum OrderType { ORDER_NONE = 0; ASC = 1; @@ -61,12 +45,13 @@ enum AlterTableType { ADD_PARTITION = 3; DROP_PARTITION = 4; SET_PROPERTY = 5; + UNSET_PROPERTY = 6; } message ColumnProto { required string name = 1; optional int32 tid = 2; - required DataType data_type = 3; + required TypeProto type = 3; } message SchemaProto { @@ -75,7 +60,7 @@ message SchemaProto { message FragmentProto { required string id = 1; - required string data_format = 2; + required string kind = 2; required bytes contents = 3; } @@ -321,6 +306,7 @@ message AlterTableDescProto { optional PartitionDescProto partition_desc = 6; optional KeyValueSetProto params = 7; optional string new_table_path = 8; + optional StringListProto unset_property_keys = 9; } message AlterColumnProto { diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestCatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestCatalogUtil.java index 0a7fd0addd..c09909a233 100644 --- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestCatalogUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestCatalogUtil.java @@ -20,6 +20,7 @@ import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.function.FunctionUtil; +import org.apache.tajo.schema.IdentifierUtil; import org.junit.Test; import java.util.Arrays; @@ -51,7 +52,7 @@ public final void testGetCanonicalName() { @Test public final void testNormalizeIdentifier() { for (int i = 0; i < sources.length; i++) { - assertEquals(normalized[i], CatalogUtil.normalizeIdentifier(sources[i])); + assertEquals(normalized[i], IdentifierUtil.normalizeIdentifier(sources[i])); } } diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java index 6aa058561e..9d6b369c82 100644 --- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java +++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java @@ -57,7 +57,9 @@ import org.apache.tajo.exception.*; import org.apache.tajo.plan.expr.AlgebraicUtil; import org.apache.tajo.plan.util.PartitionFilterAlgebraVisitor; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.type.TypeProtobufEncoder; import org.apache.tajo.util.KeyValueSet; import org.apache.thrift.TException; @@ -165,8 +167,8 @@ public final CatalogProtos.TableDescProto getTable(String databaseName, final St } if (!isPartitionKey) { - String fieldName = databaseName + CatalogConstants.IDENTIFIER_DELIMITER + tableName + - CatalogConstants.IDENTIFIER_DELIMITER + eachField.getName(); + String fieldName = databaseName + IdentifierUtil.IDENTIFIER_DELIMITER + tableName + + IdentifierUtil.IDENTIFIER_DELIMITER + eachField.getName(); TajoDataTypes.Type dataType = HiveCatalogUtil.getTajoFieldType(eachField.getType()); schemaBuilder.add(fieldName, dataType); } @@ -217,8 +219,8 @@ public final CatalogProtos.TableDescProto getTable(String databaseName, final St } } else if (BuiltinStorages.SEQUENCE_FILE.equals(dataFormat)) { - options.set(StorageConstants.SEQUENCEFILE_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter)); - options.set(StorageConstants.SEQUENCEFILE_NULL, StringEscapeUtils.escapeJava(nullFormat)); + options.set(StorageConstants.TEXT_DELIMITER, StringEscapeUtils.escapeJava(fieldDelimiter)); + options.set(StorageConstants.TEXT_NULL, StringEscapeUtils.escapeJava(nullFormat)); String serde = properties.getProperty(serdeConstants.SERIALIZATION_LIB); if (LazyBinarySerDe.class.getName().equals(serde)) { options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_BINARY_SERDE); @@ -246,8 +248,8 @@ public final CatalogProtos.TableDescProto getTable(String databaseName, final St for (int i = 0; i < partitionKeys.size(); i++) { FieldSchema fieldSchema = partitionKeys.get(i); TajoDataTypes.Type dataType = HiveCatalogUtil.getTajoFieldType(fieldSchema.getType()); - String fieldName = databaseName + CatalogConstants.IDENTIFIER_DELIMITER + tableName + - CatalogConstants.IDENTIFIER_DELIMITER + fieldSchema.getName(); + String fieldName = databaseName + IdentifierUtil.IDENTIFIER_DELIMITER + tableName + + IdentifierUtil.IDENTIFIER_DELIMITER + fieldSchema.getName(); expressionSchema.add(new Column(fieldName, dataType)); if (i > 0) { sb.append(","); @@ -421,7 +423,7 @@ public final void createTable(final CatalogProtos.TableDescProto tableDescProto) HiveCatalogStoreClientPool.HiveCatalogStoreClient client = null; TableDesc tableDesc = new TableDesc(tableDescProto); - String[] splitted = CatalogUtil.splitFQTableName(tableDesc.getName()); + String[] splitted = IdentifierUtil.splitFQTableName(tableDesc.getName()); String databaseName = splitted[0]; String tableName = splitted[1]; @@ -460,7 +462,7 @@ public final void createTable(final CatalogProtos.TableDescProto tableDescProto) for (Column eachField : columns) { cols.add(new FieldSchema(eachField.getSimpleName(), - HiveCatalogUtil.getHiveFieldType(eachField.getDataType()), "")); + HiveCatalogUtil.getHiveFieldType(eachField.getType()), "")); } sd.setCols(cols); @@ -469,7 +471,7 @@ public final void createTable(final CatalogProtos.TableDescProto tableDescProto) List partitionKeys = new ArrayList<>(); for (Column eachPartitionKey : tableDesc.getPartitionMethod().getExpressionSchema().getRootColumns()) { partitionKeys.add(new FieldSchema(eachPartitionKey.getSimpleName(), - HiveCatalogUtil.getHiveFieldType(eachPartitionKey.getDataType()), "")); + HiveCatalogUtil.getHiveFieldType(eachPartitionKey.getType()), "")); } table.setPartitionKeys(partitionKeys); } @@ -525,7 +527,7 @@ public final void createTable(final CatalogProtos.TableDescProto tableDescProto) if (StorageConstants.DEFAULT_TEXT_SERDE.equals(serde)) { sd.getSerdeInfo().setSerializationLib(LazySimpleSerDe.class.getName()); - String fieldDelimiter = tableDesc.getMeta().getProperty(StorageConstants.SEQUENCEFILE_DELIMITER, + String fieldDelimiter = tableDesc.getMeta().getProperty(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); // User can use an unicode for filed delimiter such as \u0001, \001. @@ -537,15 +539,15 @@ public final void createTable(final CatalogProtos.TableDescProto tableDescProto) StringEscapeUtils.unescapeJava(fieldDelimiter)); sd.getSerdeInfo().putToParameters(serdeConstants.FIELD_DELIM, StringEscapeUtils.unescapeJava(fieldDelimiter)); - table.getParameters().remove(StorageConstants.SEQUENCEFILE_DELIMITER); + table.getParameters().remove(StorageConstants.TEXT_DELIMITER); } else { sd.getSerdeInfo().setSerializationLib(LazyBinarySerDe.class.getName()); } - if (tableDesc.getMeta().containsProperty(StorageConstants.SEQUENCEFILE_NULL)) { + if (tableDesc.getMeta().containsProperty(StorageConstants.TEXT_NULL)) { table.putToParameters(serdeConstants.SERIALIZATION_NULL_FORMAT, - StringEscapeUtils.unescapeJava(tableDesc.getMeta().getProperty(StorageConstants.SEQUENCEFILE_NULL))); - table.getParameters().remove(StorageConstants.SEQUENCEFILE_NULL); + StringEscapeUtils.unescapeJava(tableDesc.getMeta().getProperty(StorageConstants.TEXT_NULL))); + table.getParameters().remove(StorageConstants.TEXT_NULL); } } else if (tableDesc.getMeta().getDataFormat().equalsIgnoreCase(BuiltinStorages.PARQUET)) { StorageFormatDescriptor descriptor = storageFormatFactory.get(IOConstants.PARQUET); @@ -605,7 +607,7 @@ public void alterTable(final CatalogProtos.AlterTableDescProto alterTableDescPro throws DuplicateTableException, DuplicateColumnException, DuplicatePartitionException, UndefinedPartitionException { - final String[] split = CatalogUtil.splitFQTableName(alterTableDescProto.getTableName()); + final String[] split = IdentifierUtil.splitFQTableName(alterTableDescProto.getTableName()); if (split.length == 1) { throw new IllegalArgumentException("alterTable() requires a qualified table name, but it is \"" @@ -655,6 +657,9 @@ public void alterTable(final CatalogProtos.AlterTableDescProto alterTableDescPro case SET_PROPERTY: // TODO - not implemented yet break; + case UNSET_PROPERTY: + // TODO - not implemented yet + break; default: //TODO } @@ -713,7 +718,7 @@ private void addNewColumn(String databaseName, String tableName, CatalogProtos.C Table table = client.getHiveClient().getTable(databaseName, tableName); List columns = table.getSd().getCols(); columns.add(new FieldSchema(columnProto.getName(), - HiveCatalogUtil.getHiveFieldType(columnProto.getDataType()), "")); + HiveCatalogUtil.getHiveFieldType(TypeProtobufEncoder.decode(columnProto.getType())), "")); client.getHiveClient().alter_table(databaseName, tableName, table); @@ -812,8 +817,8 @@ public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName for (int i = 0; i < partitionKeys.size(); i++) { FieldSchema fieldSchema = partitionKeys.get(i); TajoDataTypes.Type dataType = HiveCatalogUtil.getTajoFieldType(fieldSchema.getType()); - String fieldName = databaseName + CatalogConstants.IDENTIFIER_DELIMITER + tableName + - CatalogConstants.IDENTIFIER_DELIMITER + fieldSchema.getName(); + String fieldName = databaseName + IdentifierUtil.IDENTIFIER_DELIMITER + tableName + + IdentifierUtil.IDENTIFIER_DELIMITER + fieldSchema.getName(); expressionSchema.add(new Column(fieldName, dataType)); if (i > 0) { sb.append(","); @@ -1041,7 +1046,7 @@ private List getPartitionsFromHiveMetaStore(String databaseN if (i > 0) { partitionName.append(File.separator); } - partitionName.append(CatalogUtil.extractSimpleName(parititonColumns.get(i).getName())); + partitionName.append(IdentifierUtil.extractSimpleName(parititonColumns.get(i).getName())); partitionName.append("="); partitionName.append(hivePartition.getValues().get(i)); } diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java index 87b391ea60..faefd2808e 100644 --- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java +++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java @@ -39,6 +39,8 @@ import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnknownDataFormatException; import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.type.Type; +import org.apache.tajo.type.TypeStringEncoder; import org.apache.thrift.TException; public class HiveCatalogUtil { @@ -82,10 +84,10 @@ public static TajoDataTypes.Type getTajoFieldType(String dataType) throws LMDNoM } } - public static String getHiveFieldType(TajoDataTypes.DataType dataType) throws LMDNoMatchedDatatypeException { - Preconditions.checkNotNull(dataType); + public static String getHiveFieldType(Type type) throws LMDNoMatchedDatatypeException { + Preconditions.checkNotNull(type); - switch (dataType.getType()) { + switch (type.kind()) { case CHAR: return serdeConstants.CHAR_TYPE_NAME; case BOOLEAN: return serdeConstants.BOOLEAN_TYPE_NAME; case INT1: return serdeConstants.TINYINT_TYPE_NAME; @@ -104,7 +106,7 @@ public static String getHiveFieldType(TajoDataTypes.DataType dataType) throws LM case DATE: return serdeConstants.DATE_TYPE_NAME; case TIMESTAMP: return serdeConstants.TIMESTAMP_TYPE_NAME; default: - throw new LMDNoMatchedDatatypeException(dataType.getType().name()); + throw new LMDNoMatchedDatatypeException(TypeStringEncoder.encode(type)); } } diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java index fdb18530dc..d1be35b8b6 100644 --- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java +++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java @@ -36,6 +36,8 @@ import org.apache.tajo.catalog.proto.CatalogProtos.PartitionKeyProto; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.datum.NullDatum; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.KeyValueSet; @@ -106,7 +108,7 @@ public void testTableUsingTextFile() throws Exception { .add("c_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, CUSTOMER), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, CUSTOMER), schema, meta, new Path(warehousePath, new Path(DB_NAME, CUSTOMER)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, CUSTOMER)); @@ -142,7 +144,7 @@ public void testTableUsingRCFileWithBinarySerde() throws Exception { .add("r_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, REGION), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, REGION), schema, meta, new Path(warehousePath, new Path(DB_NAME, REGION)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, REGION)); @@ -177,7 +179,7 @@ public void testTableUsingRCFileWithTextSerde() throws Exception { .add("r_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, REGION), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, REGION), schema, meta, new Path(warehousePath, new Path(DB_NAME, REGION)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, REGION)); @@ -216,7 +218,7 @@ public void testTableWithNullValue() throws Exception { .add("s_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, SUPPLIER), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, SUPPLIER), schema, meta, new Path(warehousePath, new Path(DB_NAME, SUPPLIER)).toUri()); store.createTable(table.getProto()); @@ -263,7 +265,7 @@ public void testAddTableByPartition() throws Exception { .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, NATION), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, NATION), schema, meta, new Path(warehousePath, new Path(DB_NAME, NATION)).toUri()); org.apache.tajo.catalog.Schema expressionSchema = SchemaBuilder.builder() @@ -515,7 +517,7 @@ public void testGetAllTableNames() throws Exception{ String[] tableNames = new String[]{"table1", "table2", "table3"}; for(String tableName : tableNames){ - TableDesc table = new TableDesc(CatalogUtil.buildFQName("default", tableName), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName("default", tableName), schema, meta, new Path(warehousePath, new Path(DB_NAME, tableName)).toUri()); store.createTable(table.getProto()); } @@ -567,7 +569,7 @@ public void testTableUsingSequenceFileWithBinarySerde() throws Exception { .add("r_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, REGION), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, REGION), schema, meta, new Path(warehousePath, new Path(DB_NAME, REGION)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, REGION)); @@ -594,6 +596,8 @@ public void testTableUsingSequenceFileWithBinarySerde() throws Exception { public void testTableUsingSequenceFileWithTextSerde() throws Exception { KeyValueSet options = new KeyValueSet(); options.set(StorageConstants.SEQUENCEFILE_SERDE, StorageConstants.DEFAULT_TEXT_SERDE); + options.set(StorageConstants.TEXT_DELIMITER, "\u0001"); + options.set(StorageConstants.TEXT_NULL, NullDatum.DEFAULT_TEXT); TableMeta meta = new TableMeta(BuiltinStorages.SEQUENCE_FILE, options); org.apache.tajo.catalog.Schema schema = SchemaBuilder.builder() @@ -602,7 +606,7 @@ public void testTableUsingSequenceFileWithTextSerde() throws Exception { .add("r_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, REGION), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, REGION), schema, meta, new Path(warehousePath, new Path(DB_NAME, REGION)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, REGION)); @@ -621,6 +625,9 @@ public void testTableUsingSequenceFileWithTextSerde() throws Exception { } assertEquals(StorageConstants.DEFAULT_TEXT_SERDE, table1.getMeta().getProperty(StorageConstants.SEQUENCEFILE_SERDE)); + assertEquals("\u0001", StringEscapeUtils.unescapeJava(table1.getMeta().getProperty(StorageConstants + .TEXT_DELIMITER))); + assertEquals(NullDatum.DEFAULT_TEXT, table1.getMeta().getProperty(StorageConstants.TEXT_NULL)); store.dropTable(DB_NAME, REGION); } @@ -640,7 +647,7 @@ public void testTableUsingParquet() throws Exception { .add("c_comment", TajoDataTypes.Type.TEXT) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, CUSTOMER), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, CUSTOMER), schema, meta, new Path(warehousePath, new Path(DB_NAME, CUSTOMER)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, CUSTOMER)); @@ -663,7 +670,7 @@ public void testTableUsingParquet() throws Exception { @Test public void testDataTypeCompatibility() throws Exception { - String tableName = CatalogUtil.normalizeIdentifier("testDataTypeCompatibility"); + String tableName = IdentifierUtil.normalizeIdentifier("testDataTypeCompatibility"); TableMeta meta = new TableMeta(BuiltinStorages.TEXT, new KeyValueSet()); @@ -681,7 +688,7 @@ public void testDataTypeCompatibility() throws Exception { .add("col11", TajoDataTypes.Type.DATE) .build(); - TableDesc table = new TableDesc(CatalogUtil.buildFQName(DB_NAME, tableName), schema, meta, + TableDesc table = new TableDesc(IdentifierUtil.buildFQName(DB_NAME, tableName), schema, meta, new Path(warehousePath, new Path(DB_NAME, tableName)).toUri()); store.createTable(table.getProto()); assertTrue(store.existTable(DB_NAME, tableName)); diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java index df40b9b751..6583d4eb01 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java @@ -49,6 +49,7 @@ import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.ReturnState; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringListResponse; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.util.NetUtils; import org.apache.tajo.util.Pair; import org.apache.tajo.util.TUtil; @@ -441,7 +442,7 @@ public ReturnState updateTableStats(RpcController controller, UpdateTableStatsPr @Override public ReturnState alterTable(RpcController controller, AlterTableDescProto proto) { - String [] split = CatalogUtil.splitTableName(proto.getTableName()); + String [] split = IdentifierUtil.splitTableName(proto.getTableName()); if (linkedMetadataManager.existsDatabase(split[0])) { return errInsufficientPrivilege("alter a table in database '" + split[0] + "'"); @@ -666,7 +667,7 @@ public GetFunctionsResponse getFunctions(RpcController controller, @Override public ReturnState createTable(RpcController controller, TableDescProto request) { - String [] splitted = CatalogUtil.splitFQTableName(request.getTableName()); + String [] splitted = IdentifierUtil.splitFQTableName(request.getTableName()); String dbName = splitted[0]; String tbName = splitted[1]; @@ -683,7 +684,7 @@ public ReturnState createTable(RpcController controller, TableDescProto request) try { store.createTable(request); LOG.info(String.format("relation \"%s\" is added to the catalog (%s)", - CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddress)); + IdentifierUtil.getCanonicalTableName(dbName, tbName), bindAddress)); return OK; } catch (Throwable t) { @@ -713,7 +714,7 @@ public ReturnState dropTable(RpcController controller, TableIdentifierProto requ try { store.dropTable(dbName, tbName); LOG.info(String.format("relation \"%s\" is deleted from the catalog (%s)", - CatalogUtil.getCanonicalTableName(dbName, tbName), bindAddress)); + IdentifierUtil.getCanonicalTableName(dbName, tbName), bindAddress)); return OK; diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java index cb071aa783..1cf3e64c09 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/AbstractTableDescriptor.java @@ -21,6 +21,8 @@ import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.proto.CatalogProtos.*; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto; +import org.apache.tajo.schema.IdentifierUtil; +import org.apache.tajo.type.TypeFactory; abstract class AbstractTableDescriptor implements TableDescriptor { @@ -40,13 +42,7 @@ protected SchemaProto getSchemaProto() { columnBuilder = ColumnProto.newBuilder(); columnBuilder.setName(columnDescriptor.getName().toLowerCase()); - if (columnDescriptor.getLength() > 0) { - columnBuilder.setDataType(CatalogUtil.newDataTypeWithLen(columnDescriptor.getType(), - columnDescriptor.getLength())); - } else { - columnBuilder.setDataType(CatalogUtil.newSimpleDataType(columnDescriptor.getType())); - } - + columnBuilder.setType(TypeFactory.create(columnDescriptor.getType()).getProto()); schemaBuilder.addFields(columnBuilder.build()); } @@ -71,7 +67,7 @@ protected TableStatsProto getTableStatsProto() { public TableDescProto getTableDescription() { TableDescProto.Builder tableBuilder = TableDescProto.newBuilder(); - tableBuilder.setTableName(CatalogUtil.buildFQName(dictionary.getSystemDatabaseName(), getTableNameString())); + tableBuilder.setTableName(IdentifierUtil.buildFQName(dictionary.getSystemDatabaseName(), getTableNameString())); tableBuilder.setPath(dictionary.getTablePath()); tableBuilder.setSchema(CatalogUtil.getQualfiedSchema( diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java index d9008aa1b7..ea608ebfca 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/dictionary/InfoSchemaMetadataDictionary.java @@ -22,9 +22,9 @@ import java.util.Collections; import java.util.List; +import org.apache.tajo.BuiltinStorages; import org.apache.tajo.exception.UndefinedTableException; import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.proto.CatalogProtos.DataFormat; public class InfoSchemaMetadataDictionary { private static final String DATABASE_NAME = "information_schema"; @@ -129,6 +129,6 @@ public boolean existTable(String tableName) { } protected String getTablePath() { - return DataFormat.SYSTEM.name().toUpperCase(); + return BuiltinStorages.SYSTEM; } } diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java index 88fabe2b8a..c37889a909 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java @@ -31,12 +31,15 @@ import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.proto.CatalogProtos.*; -import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.exception.*; import org.apache.tajo.plan.expr.AlgebraicUtil; import org.apache.tajo.plan.util.PartitionFilterAlgebraVisitor; +import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; +import org.apache.tajo.schema.IdentifierUtil; +import org.apache.tajo.type.TypeProtobufEncoder; +import org.apache.tajo.type.TypeStringEncoder; import org.apache.tajo.util.JavaResourceUtil; import org.apache.tajo.util.Pair; @@ -49,6 +52,8 @@ import static org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto.AlterTablespaceCommand; import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueProto; import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto; +import static org.apache.tajo.schema.IdentifierUtil.extractQualifier; +import static org.apache.tajo.schema.IdentifierUtil.extractSimpleName; public abstract class AbstractDBStore extends CatalogConstants implements CatalogStore { protected final Log LOG = LogFactory.getLog(getClass()); @@ -756,7 +761,7 @@ public void createTable(final CatalogProtos.TableDescProto table) PreparedStatement pstmt = null; ResultSet res = null; - String[] splitted = CatalogUtil.splitTableName(table.getTableName()); + final String[] splitted = IdentifierUtil.splitTableName(table.getTableName()); if (splitted.length == 1) { throw new TajoInternalError( "createTable() requires a qualified table name, but it is '" + table.getTableName() + "'"); @@ -811,9 +816,9 @@ public void createTable(final CatalogProtos.TableDescProto table) String colSql = "INSERT INTO " + TB_COLUMNS + - // 1 2 3 4 5 6 - " (TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH)" + - " VALUES(?, ?, ?, ?, ?, ?) "; + // 1 2 3 4 + " (TID, COLUMN_NAME, ORDINAL_POSITION, DATA_TYPE)" + + " VALUES(?, ?, ?, ?) "; if (LOG.isDebugEnabled()) { LOG.debug(colSql); @@ -822,15 +827,12 @@ public void createTable(final CatalogProtos.TableDescProto table) pstmt = conn.prepareStatement(colSql); for (int i = 0; i < table.getSchema().getFieldsCount(); i++) { ColumnProto col = table.getSchema().getFields(i); - TajoDataTypes.DataType dataType = col.getDataType(); + org.apache.tajo.type.Type type = TypeProtobufEncoder.decode(col.getType()); pstmt.setInt(1, tableId); - pstmt.setString(2, CatalogUtil.extractSimpleName(col.getName())); + pstmt.setString(2, extractSimpleName(col.getName())); pstmt.setInt(3, i); - // the default number of nested fields is 0. - pstmt.setInt(4, dataType.hasNumNestedFields() ? dataType.getNumNestedFields() : 0); - pstmt.setString(5, dataType.getType().name()); - pstmt.setInt(6, (col.getDataType().hasLength() ? col.getDataType().getLength() : 0)); + pstmt.setString(4, TypeStringEncoder.encode(type)); pstmt.addBatch(); pstmt.clearParameters(); } @@ -911,7 +913,7 @@ public void updateTableStats(final CatalogProtos.UpdateTableStatsProto statsProt PreparedStatement pstmt = null; ResultSet res = null; - String[] splitted = CatalogUtil.splitTableName(statsProto.getTableName()); + String[] splitted = IdentifierUtil.splitTableName(statsProto.getTableName()); if (splitted.length == 1) { throw new IllegalArgumentException("updateTableStats() requires a qualified table name, but it is \"" + statsProto.getTableName() + "\"."); @@ -978,7 +980,7 @@ public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) DuplicatePartitionException, UndefinedPartitionException, UndefinedColumnException, UndefinedTableException, UndefinedPartitionMethodException, AmbiguousTableException { - String[] splitted = CatalogUtil.splitTableName(alterTableDescProto.getTableName()); + String[] splitted = IdentifierUtil.splitTableName(alterTableDescProto.getTableName()); if (splitted.length == 1) { throw new IllegalArgumentException("alterTable() requires a qualified table name, but it is \"" + alterTableDescProto.getTableName() + "\"."); @@ -993,7 +995,7 @@ public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) switch (alterTableDescProto.getAlterTableType()) { case RENAME_TABLE: - String simpleNewTableName = CatalogUtil.extractSimpleName(alterTableDescProto.getNewTableName()); + String simpleNewTableName = extractSimpleName(alterTableDescProto.getNewTableName()); if (existTable(databaseName, simpleNewTableName)) { throw new DuplicateTableException(alterTableDescProto.getNewTableName()); } @@ -1036,6 +1038,9 @@ public void alterTable(CatalogProtos.AlterTableDescProto alterTableDescProto) case SET_PROPERTY: setProperties(tableId, alterTableDescProto.getParams()); break; + case UNSET_PROPERTY: + unsetProperties(tableId, alterTableDescProto.getUnsetPropertyKeys()); + break; default: } } @@ -1105,6 +1110,38 @@ private void setProperties(final int tableId, final KeyValueSetProto properties) } } + private void unsetProperties(final int tableId, final PrimitiveProtos.StringListProto propertyKeys) { + final String deleteSql = "DELETE FROM " + TB_OPTIONS + " WHERE TID=? AND KEY_=?"; + + Connection conn; + PreparedStatement pstmt = null; + + Map oldProperties = getTableOptions(tableId); + + try { + conn = getConnection(); + conn.setAutoCommit(false); + + for (String key : propertyKeys.getValuesList()) { + if (oldProperties.containsKey(key)) { + // unset property + pstmt = conn.prepareStatement(deleteSql); + + pstmt.setInt(1, tableId); + pstmt.setString(2, key); + pstmt.executeUpdate(); + pstmt.close(); + } + } + + conn.commit(); + } catch (Throwable sqlException) { + throw new TajoInternalError(sqlException); + } finally { + CatalogUtil.closeQuietly(pstmt); + } + } + private void renameExternalTable(final int tableId, final String tableName) { final String updtaeRenameTableSql = "UPDATE " + TB_TABLES + " SET " + COL_TABLES_NAME + " = ? " + " WHERE TID = ?"; @@ -1149,13 +1186,13 @@ private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProt throws UndefinedColumnException, AmbiguousTableException { final String selectColumnSql = - "SELECT COLUMN_NAME, DATA_TYPE, TYPE_LENGTH, ORDINAL_POSITION, NESTED_FIELD_NUM from " + TB_COLUMNS + + "SELECT COLUMN_NAME, DATA_TYPE, ORDINAL_POSITION from " + TB_COLUMNS + " WHERE " + COL_TABLES_PK + " = ?" + " AND COLUMN_NAME = ?" ; final String deleteColumnNameSql = "DELETE FROM " + TB_COLUMNS + " WHERE TID = ? AND COLUMN_NAME = ?"; final String insertNewColumnSql = "INSERT INTO " + TB_COLUMNS + - " (TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH) VALUES(?, ?, ?, ?, ?, ?) "; + " (TID, COLUMN_NAME, ORDINAL_POSITION, DATA_TYPE) VALUES(?, ?, ?, ?) "; if (LOG.isDebugEnabled()) { LOG.debug(selectColumnSql); @@ -1172,13 +1209,13 @@ private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProt conn = getConnection(); conn.setAutoCommit(false); - String tableName = CatalogUtil.extractQualifier(alterColumnProto.getOldColumnName()); - String simpleOldColumnName = CatalogUtil.extractSimpleName(alterColumnProto.getOldColumnName()); - String simpleNewColumnName = CatalogUtil.extractSimpleName(alterColumnProto.getNewColumnName()); + String tableName = extractQualifier(alterColumnProto.getOldColumnName()); + String simpleOldColumnName = extractSimpleName(alterColumnProto.getOldColumnName()); + String simpleNewColumnName = extractSimpleName(alterColumnProto.getNewColumnName()); - if (!tableName.equals(CatalogUtil.extractQualifier(alterColumnProto.getNewColumnName()))) { + if (!tableName.equals(extractQualifier(alterColumnProto.getNewColumnName()))) { throw new AmbiguousTableException( - tableName + ", " + CatalogUtil.extractQualifier(alterColumnProto.getNewColumnName())); + tableName + ", " + extractQualifier(alterColumnProto.getNewColumnName())); } //SELECT COLUMN @@ -1188,15 +1225,15 @@ private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProt resultSet = pstmt.executeQuery(); CatalogProtos.ColumnProto columnProto = null; + String typeStr; int ordinalPosition = 0; - int nestedFieldNum = 0; if (resultSet.next()) { columnProto = resultToColumnProto(resultSet); //NOTE ==> Setting new column Name columnProto = columnProto.toBuilder().setName(alterColumnProto.getNewColumnName()).build(); ordinalPosition = resultSet.getInt("ORDINAL_POSITION"); - nestedFieldNum = resultSet.getInt("NESTED_FIELD_NUM"); + typeStr = TypeStringEncoder.encode(TypeProtobufEncoder.decode(columnProto.getType())); } else { throw new UndefinedColumnException(alterColumnProto.getOldColumnName()); } @@ -1217,9 +1254,7 @@ private void renameColumn(final int tableId, final CatalogProtos.AlterColumnProt pstmt.setInt(1, tableId); pstmt.setString(2, simpleNewColumnName); pstmt.setInt(3, ordinalPosition); - pstmt.setInt(4, nestedFieldNum); - pstmt.setString(5, columnProto.getDataType().getType().name()); - pstmt.setInt(6, (columnProto.getDataType().hasLength() ? columnProto.getDataType().getLength() : 0)); + pstmt.setString(4, typeStr); pstmt.executeUpdate(); conn.commit(); @@ -1243,7 +1278,7 @@ private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) th final String insertNewColumnSql = "INSERT INTO " + TB_COLUMNS + - " (TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH) VALUES(?, ?, ?, ?, ?, ?) "; + " (TID, COLUMN_NAME, ORDINAL_POSITION, DATA_TYPE) VALUES(?, ?, ?, ?) "; final String columnCountSql = "SELECT MAX(ORDINAL_POSITION) AS POSITION FROM " + TB_COLUMNS + " WHERE TID = ?"; @@ -1251,7 +1286,7 @@ private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) th conn = getConnection(); pstmt = conn.prepareStatement(existColumnSql); pstmt.setInt(1, tableId); - pstmt.setString(2, CatalogUtil.extractSimpleName(columnProto.getName())); + pstmt.setString(2, extractSimpleName(columnProto.getName())); resultSet = pstmt.executeQuery(); if (resultSet.next()) { @@ -1271,15 +1306,13 @@ private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) th pstmt.close(); resultSet = null; - TajoDataTypes.DataType dataType = columnProto.getDataType(); + org.apache.tajo.type.Type type = TypeProtobufEncoder.decode(columnProto.getType()); pstmt = conn.prepareStatement(insertNewColumnSql); pstmt.setInt(1, tableId); - pstmt.setString(2, CatalogUtil.extractSimpleName(columnProto.getName())); + pstmt.setString(2, extractSimpleName(columnProto.getName())); pstmt.setInt(3, position + 1); - pstmt.setInt(4, dataType.hasNumNestedFields() ? dataType.getNumNestedFields() : 0); - pstmt.setString(5, dataType.getType().name()); - pstmt.setInt(6, (columnProto.getDataType().hasLength() ? columnProto.getDataType().getLength() : 0)); + pstmt.setString(4, TypeStringEncoder.encode(type)); pstmt.executeUpdate(); } catch (SQLException sqlException) { @@ -1600,7 +1633,7 @@ public CatalogProtos.TableDescProto getTable(String databaseName, String tableNa } int tableId = res.getInt(1); - tableBuilder.setTableName(CatalogUtil.buildFQName(databaseName, res.getString(2).trim())); + tableBuilder.setTableName(IdentifierUtil.buildFQName(databaseName, res.getString(2).trim())); TableType tableType = TableType.valueOf(res.getString(3)); if (tableType == TableType.EXTERNAL) { tableBuilder.setIsExternal(true); @@ -1618,7 +1651,7 @@ public CatalogProtos.TableDescProto getTable(String databaseName, String tableNa // Geting Column Descriptions ////////////////////////////////////////// CatalogProtos.SchemaProto.Builder schemaBuilder = CatalogProtos.SchemaProto.newBuilder(); - sql = "SELECT COLUMN_NAME, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH from " + TB_COLUMNS + + sql = "SELECT COLUMN_NAME, DATA_TYPE from " + TB_COLUMNS + " WHERE " + COL_TABLES_PK + " = ? ORDER BY ORDINAL_POSITION ASC"; if (LOG.isDebugEnabled()) { @@ -1847,7 +1880,7 @@ public List getAllColumns() { try { String sql = - "SELECT TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH FROM " + TB_COLUMNS + + "SELECT TID, COLUMN_NAME, ORDINAL_POSITION, DATA_TYPE FROM " + TB_COLUMNS + " ORDER BY TID ASC, ORDINAL_POSITION ASC"; conn = getConnection(); @@ -1860,21 +1893,9 @@ public List getAllColumns() { String databaseName = getDatabaseNameOfTable(conn, tid); String tableName = getTableName(conn, tid); builder.setTid(tid); - builder.setName(CatalogUtil.buildFQName(databaseName, tableName, resultSet.getString("COLUMN_NAME"))); - - int nestedFieldNum = resultSet.getInt("NESTED_FIELD_NUM"); - - Type type = getDataType(resultSet.getString("DATA_TYPE").trim()); - int typeLength = resultSet.getInt("TYPE_LENGTH"); - - if (nestedFieldNum > 0) { - builder.setDataType(CatalogUtil.newRecordType(nestedFieldNum)); - } else if (typeLength > 0) { - builder.setDataType(CatalogUtil.newDataTypeWithLen(type, typeLength)); - } else { - builder.setDataType(CatalogUtil.newSimpleDataType(type)); - } - + builder.setName(IdentifierUtil.buildFQName(databaseName, tableName, resultSet.getString("COLUMN_NAME"))); + org.apache.tajo.type.Type type = TypeStringEncoder.decode(resultSet.getString("DATA_TYPE").trim()); + builder.setType(type.getProto()); columns.add(builder.build()); } } catch (SQLException se) { @@ -2484,7 +2505,7 @@ public void createIndex(final IndexDescProto proto) PreparedStatement pstmt = null; final String databaseName = proto.getTableIdentifier().getDatabaseName(); - final String tableName = CatalogUtil.extractSimpleName(proto.getTableIdentifier().getTableName()); + final String tableName = extractSimpleName(proto.getTableIdentifier().getTableName()); try { @@ -2529,7 +2550,7 @@ public void createIndex(final IndexDescProto proto) // Since the key columns are always sorted in order of their occurrence position in the relation schema, // the concatenated name can be uniquely identified. columnNamesBuilder.append(columnSpec.getSortKey().getSimpleName()).append(","); - dataTypesBuilder.append(columnSpec.getSortKey().getDataType().getType().name()).append(","); + dataTypesBuilder.append(columnSpec.getSortKey().getDataType().getType().name()).append("|"); ordersBuilder.append(columnSpec.isAscending()).append(","); nullOrdersBuilder.append(columnSpec.isNullsFirst()).append(","); } @@ -2576,7 +2597,7 @@ public void dropIndex(String databaseName, final String indexName) pstmt.setString(2, indexName); ResultSet res = pstmt.executeQuery(); if (!res.next()) { - throw new UndefinedIndexException(CatalogUtil.buildFQName(databaseName, indexName)); + throw new UndefinedIndexException(IdentifierUtil.buildFQName(databaseName, indexName)); } pstmt.close(); res.close(); @@ -2676,7 +2697,7 @@ public IndexDescProto getIndexByName(String databaseName, final String indexName IndexDescProto.Builder builder = IndexDescProto.newBuilder(); String tableName = getTableName(conn, res.getInt(COL_TABLES_PK)); builder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName)); - resultToIndexDescProtoBuilder(CatalogUtil.buildFQName(databaseName, tableName), builder, res); + resultToIndexDescProtoBuilder(IdentifierUtil.buildFQName(databaseName, tableName), builder, res); try { builder.setTargetRelationSchema(getTable(databaseName, tableName).getSchema()); @@ -2727,7 +2748,7 @@ public IndexDescProto getIndexByColumns(String databaseName, String tableName, S } IndexDescProto.Builder builder = IndexDescProto.newBuilder(); - resultToIndexDescProtoBuilder(CatalogUtil.buildFQName(databaseName, tableName), builder, res); + resultToIndexDescProtoBuilder(IdentifierUtil.buildFQName(databaseName, tableName), builder, res); builder.setTableIdentifier(CatalogUtil.buildTableIdentifier(databaseName, tableName)); builder.setTargetRelationSchema(tableDescProto.getSchema()); proto = builder.build(); @@ -2897,14 +2918,14 @@ private void resultToIndexDescProtoBuilder(final String qualifier, builder.setIndexPath(res.getString("path")); String[] columnNames, dataTypes, orders, nullOrders; columnNames = res.getString("column_names").trim().split(","); - dataTypes = res.getString("data_types").trim().split(","); + dataTypes = res.getString("data_types").trim().split("\\|"); orders = res.getString("orders").trim().split(","); nullOrders = res.getString("null_orders").trim().split(","); int columnNum = columnNames.length; for (int i = 0; i < columnNum; i++) { SortSpecProto.Builder colSpecBuilder = SortSpecProto.newBuilder(); - colSpecBuilder.setColumn(ColumnProto.newBuilder().setName(CatalogUtil.buildFQName(qualifier, columnNames[i])) - .setDataType(CatalogUtil.newSimpleDataType(getDataType(dataTypes[i]))).build()); + colSpecBuilder.setColumn(ColumnProto.newBuilder().setName(IdentifierUtil.buildFQName(qualifier, columnNames[i])) + .setType(TypeStringEncoder.decode(dataTypes[i]).getProto()).build()); colSpecBuilder.setAscending(orders[i].equals("true")); colSpecBuilder.setNullFirst(nullOrders[i].equals("true")); builder.addKeySortSpecs(colSpecBuilder.build()); @@ -2916,20 +2937,8 @@ private void resultToIndexDescProtoBuilder(final String qualifier, private ColumnProto resultToColumnProto(final ResultSet res) throws SQLException { ColumnProto.Builder builder = ColumnProto.newBuilder(); builder.setName(res.getString("column_name").trim()); - - int nestedFieldNum = res.getInt("NESTED_FIELD_NUM"); - - Type type = getDataType(res.getString("data_type").trim()); - int typeLength = res.getInt("type_length"); - - if (nestedFieldNum > 0) { - builder.setDataType(CatalogUtil.newRecordType(nestedFieldNum)); - } else if (typeLength > 0) { - builder.setDataType(CatalogUtil.newDataTypeWithLen(type, typeLength)); - } else { - builder.setDataType(CatalogUtil.newSimpleDataType(type)); - } - + org.apache.tajo.type.Type type = TypeStringEncoder.decode(res.getString("data_type").trim()); + builder.setType(type.getProto()); return builder.build(); } diff --git a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml index 96100e8430..70ef436bec 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml +++ b/tajo-catalog/tajo-catalog-server/src/main/resources/schemas/derby/derby.xml @@ -19,6 +19,7 @@ + +# Apache Tajo™ 0.11.3 Release Announcement + +The Apache Tajo team is pleased to announce the release of Apache Tajo™ 0.11.3! +Apache Tajo™ is a big data warehouse system on various data sources. It provides distributed and scalable SQL analytical processing on Apache Hadoop™. + +The release is available for immediate download: + + * http://tajo.apache.org/downloads.html + +This is a minor release. In this release, we fixed 5 bugs, and temporarily disabled the 'NOT IN' predicate. + +### Some of Highlights + * Fix incorrect DateTime and remove hard coded tests (TAJO-2110) + * Fix invalid sort result when sort key columns contain non-ascii values. (TAJO-2077) + * Fix invalid join result when join key columns contain nulls. (TAJO-2135) + * Fix an empty reason and stacktrace of TajoInternalError (TAJO-2140) + * Fix race condition in task history writer (TAJO-2143) + +For a complete list of new features and fixed problems, please see the release notes: + + * http://tajo.apache.org/releases/0.11.3/relnotes.html diff --git a/tajo-project/src/site/resources/releases/0.11.3/relnotes.html b/tajo-project/src/site/resources/releases/0.11.3/relnotes.html new file mode 100644 index 0000000000..e914da519a --- /dev/null +++ b/tajo-project/src/site/resources/releases/0.11.3/relnotes.html @@ -0,0 +1,35 @@ + + +Release Notes - Tajo - Version 0.11.3 + + + +

Release Notes - Tajo - Version 0.11.3

+ +

Bug +

+
    +
  • [TAJO-2110] - Fix incorrect DateTime and remove hard coded tests +
  • +
  • [TAJO-2119] - Invalid sort result when sort key columns contain non-ascii values +
  • +
  • [TAJO-2135] - Invalid join result when join key columns contain nulls +
  • +
  • [TAJO-2140] - TajoInternalError does not contains reason stack trace +
  • +
  • [TAJO-2143] - Fix race condition in task history writer +
  • +
+ +

Task +

+
    +
  • [TAJO-2142] - Temporarily disable NOT IN subquery +
  • +
  • [TAJO-2145] - Error codes based on errno.h need prefix +
  • +
+ + + + diff --git a/tajo-project/src/site/site.xml b/tajo-project/src/site/site.xml index f36e810165..20e34eb918 100644 --- a/tajo-project/src/site/site.xml +++ b/tajo-project/src/site/site.xml @@ -71,18 +71,33 @@ function addListener(element, type, callback) { } function ga_event_load() { + var dn_src_0_11_3 = document.getElementById('tajo-0.11.3-src'); + var dn_bin_0_11_3 = document.getElementById('tajo-0.11.3-bin'); + var dn_jdbc_0_11_3 = document.getElementById('tajo-jdbc-0.11.3'); + + addListener(dn_src_0_11_3, 'click', function() { + ga('send', 'event', 'tajo-0.11.3-src', 'click', 'tajo-0.11.3-src'); + }); + addListener(dn_bin_0_11_3, 'click', function() { + ga('send', 'event', 'tajo-0.11.3-bin', 'click', 'tajo-0.11.3-bin'); + }); + addListener(dn_jdbc_0_11_3, 'click', function() { + ga('send', 'event', 'tajo-jdbc-0.11.3', 'click', 'tajo-jdbc-0.11.3'); + }); + + var dn_src_0_11_2 = document.getElementById('tajo-0.11.2-src'); var dn_bin_0_11_2 = document.getElementById('tajo-0.11.2-bin'); var dn_jdbc_0_11_2 = document.getElementById('tajo-jdbc-0.11.2'); addListener(dn_src_0_11_2, 'click', function() { - ga('send', 'event', 'tajo-0.11.2-src', 'click', 'tajo-0.11.1-src'); + ga('send', 'event', 'tajo-0.11.2-src', 'click', 'tajo-0.11.2-src'); }); addListener(dn_bin_0_11_2, 'click', function() { - ga('send', 'event', 'tajo-0.11.2-bin', 'click', 'tajo-0.11.1-bin'); + ga('send', 'event', 'tajo-0.11.2-bin', 'click', 'tajo-0.11.2-bin'); }); addListener(dn_jdbc_0_11_2, 'click', function() { - ga('send', 'event', 'tajo-jdbc-0.11.2', 'click', 'tajo-jdbc-0.11.1'); + ga('send', 'event', 'tajo-jdbc-0.11.2', 'click', 'tajo-jdbc-0.11.2'); }); @@ -185,7 +200,7 @@ window.onload = ga_event_load; - + diff --git a/tajo-sql-parser/pom.xml b/tajo-sql-parser/pom.xml index cba8af5934..956e410ad9 100644 --- a/tajo-sql-parser/pom.xml +++ b/tajo-sql-parser/pom.xml @@ -52,6 +52,11 @@ + + + **/*.tokens + + maven-deploy-plugin diff --git a/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLLexer.g4 b/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLLexer.g4 index 2bb3582d84..e67b435abe 100644 --- a/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLLexer.g4 +++ b/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLLexer.g4 @@ -259,7 +259,6 @@ LESS : L E S S; LIST : L I S T; LOCATION : L O C A T I O N; -MAP : M A P; MAX : M A X; MAXVALUE : M A X V A L U E; MICROSECONDS : M I C R O S E C O N D S; @@ -288,7 +287,6 @@ QUARTER : Q U A R T E R; RANGE : R A N G E; RANK : R A N K; -RECORD : R E C O R D; REGEXP : R E G E X P; RENAME : R E N A M E; REPAIR : R E P A I R; @@ -320,6 +318,7 @@ TRUNCATE : T R U N C A T E; UNBOUNDED : U N B O U N D E D; UNKNOWN : U N K N O W N; +UNSET : U N S E T; VALUES : V A L U E S; VAR_SAMP : V A R UNDERLINE S A M P; @@ -383,7 +382,10 @@ VARBINARY : V A R B I N A R Y; BLOB : B L O B; BYTEA : B Y T E A; // alias for BLOB -INET4 : I N E T '4'; +// complex types +ARRAY : A R R A Y; +MAP : M A P; +RECORD : R E C O R D; // Operators Similar_To : '~'; diff --git a/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLParser.g4 b/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLParser.g4 index 190825863a..8b5d36e189 100644 --- a/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLParser.g4 +++ b/tajo-sql-parser/src/main/antlr4/org/apache/tajo/parser/sql/SQLParser.g4 @@ -211,7 +211,7 @@ truncate_table_statement /* =============================================================================== - 11.21 + 11.21 =============================================================================== */ @@ -441,9 +441,7 @@ predefined_type | datetime_type | bit_type | binary_type - | network_type - | record_type - | map_type + | complex_type ; character_string_type @@ -535,8 +533,14 @@ binary_type | VARBINARY type_length? ; -network_type - : INET4 +complex_type + : array_type + | record_type + | map_type + ; + +array_type + : ARRAY LTH data_type GTH ; record_type @@ -765,10 +769,6 @@ factor : (sign)? numeric_primary ; -array - : LEFT_PAREN numeric_value_expression (COMMA numeric_value_expression )* RIGHT_PAREN - ; - numeric_primary : value_expression_primary (CAST_EXPRESSION cast_target)* | numeric_value_function @@ -1628,6 +1628,7 @@ alter_table_statement | ALTER TABLE table_name ADD (if_not_exists)? PARTITION LEFT_PAREN partition_column_value_list RIGHT_PAREN (LOCATION path=Character_String_Literal)? | ALTER TABLE table_name DROP (if_exists)? PARTITION LEFT_PAREN partition_column_value_list RIGHT_PAREN (PURGE)? | ALTER TABLE table_name SET PROPERTY property_list + | ALTER TABLE table_name UNSET PROPERTY property_key_list | ALTER TABLE table_name REPAIR PARTITION ; @@ -1646,3 +1647,11 @@ property_list property : key=Character_String_Literal EQUAL value=Character_String_Literal ; + +property_key_list + : property_key (COMMA property_key)* + ; + +property_key + : key=Character_String_Literal + ; diff --git a/tajo-storage/pom.xml b/tajo-storage/pom.xml index 4881e2c7ad..e5f7588df6 100644 --- a/tajo-storage/pom.xml +++ b/tajo-storage/pom.xml @@ -39,6 +39,7 @@ tajo-storage-hbase tajo-storage-jdbc tajo-storage-pgsql + tajo-storage-mysql tajo-storage-s3 diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java index 51dfba53fa..057faee1f3 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/BinarySerializerDeserializer.java @@ -93,8 +93,6 @@ public int serialize(int index, Tuple tuple, OutputStream out, byte[] nullCharac break; } case BLOB: - case INET4: - case INET6: bytes = tuple.getBytes(index); length = bytes.length; out.write(bytes, 0, length); @@ -163,9 +161,6 @@ public Datum deserialize(int index, byte[] bytes, int offset, int length, byte[] bytes, offset, length); break; } - case INET4: - datum = DatumFactory.createInet4(bytes, offset, length); - break; case BLOB: datum = DatumFactory.createBlob(bytes, offset, length); break; diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java index 72ffde7296..903ca39d82 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/LazyTuple.java @@ -116,7 +116,7 @@ public Datum asDatum(int fieldId) { @Override public TajoDataTypes.Type type(int fieldId) { - return get(fieldId).type(); + return get(fieldId).kind(); } @Override diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java deleted file mode 100644 index 16477cd8ef..0000000000 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/MemoryUtil.java +++ /dev/null @@ -1,167 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.storage; - -import org.apache.tajo.datum.*; -import org.apache.tajo.util.ClassSize; - -public class MemoryUtil { - - /** Overhead for an NullDatum */ - public static final long NULL_DATUM; - - /** Overhead for an BoolDatum */ - public static final long BOOL_DATUM; - - /** Overhead for an CharDatum */ - public static final long CHAR_DATUM; - - /** Overhead for an BitDatum */ - public static final long BIT_DATUM; - - /** Overhead for an Int2Datum */ - public static final long INT2_DATUM; - - /** Overhead for an Int4Datum */ - public static final long INT4_DATUM; - - /** Overhead for an Int8Datum */ - public static final long INT8_DATUM; - - /** Overhead for an Float4Datum */ - public static final long FLOAT4_DATUM; - - /** Overhead for an Float8Datum */ - public static final long FLOAT8_DATUM; - - /** Overhead for an TextDatum */ - public static final long TEXT_DATUM; - - /** Overhead for an BlobDatum */ - public static final long BLOB_DATUM; - - /** Overhead for an DateDatum */ - public static final long DATE_DATUM; - - /** Overhead for an TimeDatum */ - public static final long TIME_DATUM; - - /** Overhead for an TimestampDatum */ - public static final long TIMESTAMP_DATUM; - - static { - NULL_DATUM = ClassSize.estimateBase(NullDatum.class, false); - - CHAR_DATUM = ClassSize.estimateBase(CharDatum.class, false); - - BOOL_DATUM = ClassSize.estimateBase(BooleanDatum.class, false); - - BIT_DATUM = ClassSize.estimateBase(BitDatum.class, false); - - INT2_DATUM = ClassSize.estimateBase(Int2Datum.class, false); - - INT4_DATUM = ClassSize.estimateBase(Int4Datum.class, false); - - INT8_DATUM = ClassSize.estimateBase(Int8Datum.class, false); - - FLOAT4_DATUM = ClassSize.estimateBase(Float4Datum.class, false); - - FLOAT8_DATUM = ClassSize.estimateBase(Float8Datum.class, false); - - TEXT_DATUM = ClassSize.estimateBase(TextDatum.class, false); - - BLOB_DATUM = ClassSize.estimateBase(BlobDatum.class, false); - - DATE_DATUM = ClassSize.estimateBase(DateDatum.class, false); - - TIME_DATUM = ClassSize.estimateBase(TimeDatum.class, false); - - TIMESTAMP_DATUM = ClassSize.estimateBase(TimestampDatum.class, false); - } - - public static long calculateMemorySize(Tuple tuple) { - long total = ClassSize.OBJECT; - for (Datum datum : tuple.getValues()) { - switch (datum.type()) { - - case NULL_TYPE: - total += NULL_DATUM; - break; - - case BOOLEAN: - total += BOOL_DATUM; - break; - - case BIT: - total += BIT_DATUM; - break; - - case CHAR: - total += CHAR_DATUM + datum.size(); - break; - - case INT1: - case INT2: - total += INT2_DATUM; - break; - - case INT4: - total += INT4_DATUM; - break; - - case INT8: - total += INT8_DATUM; - break; - - case FLOAT4: - total += FLOAT4_DATUM; - break; - - case FLOAT8: - total += FLOAT4_DATUM; - break; - - case TEXT: - total += TEXT_DATUM + datum.size(); - break; - - case BLOB: - total += BLOB_DATUM + datum.size(); - break; - - case DATE: - total += DATE_DATUM; - break; - - case TIME: - total += TIME_DATUM; - break; - - case TIMESTAMP: - total += TIMESTAMP_DATUM; - break; - - default: - break; - } - } - - return total; - } -} diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java index 8ca55cc112..c8474b1d21 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/RowStoreUtil.java @@ -149,11 +149,6 @@ public Tuple toTuple(byte [] bytes) { tuple.put(i, DatumFactory.createBlob(_bytes)); break; - case INET4: - byte [] _ipv4 = new byte[4]; - bb.get(_ipv4); - tuple.put(i, DatumFactory.createInet4(_ipv4)); - break; default: throw new TajoRuntimeException( new UnsupportedException("data type '" + type.getType().name() + "'")); @@ -249,13 +244,6 @@ public byte[] toBytes(Tuple tuple) { bb.putInt(bytes.length); bb.put(bytes); break; - case INET4: - byte[] ipBytes = tuple.getBytes(i); - bb.put(ipBytes); - break; - case INET6: - bb.put(tuple.getBytes(i)); - break; default: throw new TajoRuntimeException( new UnsupportedException("data type '" + col.getDataType().getType().name() + "'")); @@ -314,10 +302,6 @@ private int estimateTupleDataSize(Tuple tuple) { case BLOB: size += (4 + tuple.getBytes(i).length); break; - case INET4: - case INET6: - size += tuple.getBytes(i).length; - break; default: throw new TajoRuntimeException( new UnsupportedException("data type '" + col.getDataType().getType().name() + "'")); diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java index 00e6d75a12..64b78d6bfd 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/Tablespace.java @@ -99,7 +99,7 @@ public String toString() { return name + "=" + uri.toString(); } - public abstract long getTableVolume(TableDesc table, Optional filter) throws UnsupportedException; + public abstract long getTableVolume(TableDesc table, Optional filter); /** * if {@link StorageProperty#isArbitraryPathAllowed} is true, @@ -115,23 +115,27 @@ public URI getRootUri() { /** * Get Table URI * + * @param meta table meta * @param databaseName Database name * @param tableName Table name * @return Table URI */ - public abstract URI getTableUri(String databaseName, String tableName); + public abstract URI getTableUri(TableMeta meta, String databaseName, String tableName); /** * Returns the splits that will serve as input for the scan tasks. The * number of splits matches the number of regions in a table. + * * @param inputSourceId Input source identifier, which can be either relation name or execution block id * @param tableDesc The table description for the target data. + * @param requireSort The result fragments will be sorted with their paths. * @param filterCondition filter condition which can prune splits if possible * @return The list of input fragments. * @throws java.io.IOException */ public abstract List getSplits(String inputSourceId, TableDesc tableDesc, + boolean requireSort, @Nullable EvalNode filterCondition) throws IOException, TajoException; /** @@ -240,6 +244,21 @@ public synchronized SeekableScanner getSeekableScanner(TableMeta meta, Schema sc return (SeekableScanner)this.getScanner(meta, schema, FragmentConvertor.convert(conf, fragment), target); } + /** + * Returns Scanner instance. + * + * @param meta The table meta + * @param schema The input schema + * @param fragment The fragment for scanning + * @param target The output schema + * @return Scanner instance + * @throws IOException + */ + public synchronized SeekableScanner getSeekableScanner(TableMeta meta, Schema schema, Fragment fragment, + Schema target) throws IOException { + return (SeekableScanner)this.getScanner(meta, schema, fragment, target); + } + /** * Returns Appender instance. * @param queryContext Query property. @@ -262,13 +281,12 @@ public Appender getAppender(OverridableConf queryContext, if (appenderClass == null) { appenderClass = conf.getClass( String.format("tajo.storage.appender-handler.%s.class", handlerName), null, Appender.class); + if (appenderClass == null) { + throw new IOException("Undefined appender handler for " + meta.getDataFormat()); + } OldStorageManager.APPENDER_HANDLER_CACHE.put(handlerName, appenderClass); } - if (appenderClass == null) { - throw new IOException("Unknown Storage Type: " + meta.getDataFormat()); - } - appender = OldStorageManager.newAppenderInstance(appenderClass, conf, taskAttemptId, meta, schema, workDir); return appender; diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java index 88410bb2cc..9721df0259 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java @@ -32,6 +32,7 @@ import org.apache.tajo.TajoConstants; import org.apache.tajo.catalog.MetadataProvider; import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UndefinedTablespaceException; @@ -430,9 +431,9 @@ public static Optional getAnyByScheme(String scheme) { } @Override - public URI getTableURI(@Nullable String spaceName, String databaseName, String tableName) { + public URI getTableURI(@Nullable String spaceName, TableMeta meta, String databaseName, String tableName) { Tablespace space = spaceName == null ? getDefault() : getByName(spaceName); - return space.getTableUri(databaseName, tableName); + return space.getTableUri(meta, databaseName, tableName); } @Override diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java index 0bc2168ce4..ef20602f53 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TextSerializerDeserializer.java @@ -92,7 +92,6 @@ public int serialize(int index, Tuple tuple, OutputStream out, byte[] nullCharac case INT8: case FLOAT4: case FLOAT8: - case INET4: case DATE: case INTERVAL: bytes = tuple.getTextBytes(index); @@ -104,7 +103,6 @@ public int serialize(int index, Tuple tuple, OutputStream out, byte[] nullCharac length = bytes.length; out.write(bytes); break; - case INET6: case BLOB: bytes = Base64.encodeBase64(tuple.getBytes(index), false); length = bytes.length; @@ -201,10 +199,6 @@ public Datum deserialize(int index, byte[] bytes, int offset, int length, byte[] } break; } - case INET4: - datum = isNull(bytes, offset, length, nullCharacters) ? NullDatum.get() - : DatumFactory.createInet4(new String(bytes, offset, length)); - break; case BLOB: { if (isNull(bytes, offset, length, nullCharacters)) { datum = NullDatum.get(); diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/BuiltinFragmentKinds.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/BuiltinFragmentKinds.java new file mode 100644 index 0000000000..cc7eea55fb --- /dev/null +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/BuiltinFragmentKinds.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.fragment; + +public class BuiltinFragmentKinds { + public static final String FILE = "FILE"; + public static final String HBASE = "HBASE"; + public static final String JDBC = "JDBC"; + public static final String HTTP = "EXAMPLE-HTTP"; +} diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java index ac43197296..a8de4abad7 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/Fragment.java @@ -18,22 +18,150 @@ package org.apache.tajo.storage.fragment; -import org.apache.tajo.common.ProtoObject; +import com.google.common.collect.ImmutableList; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import java.net.URI; -public interface Fragment extends ProtoObject { +/** + * The Fragment is similar to the split in MapReduce. + * For distributed processing of a a single large table, + * it contains the information of which part of data will be processed by each task. + * + * @param type of fragment key. It should implement the Comparable interface. + */ +public abstract class Fragment implements Comparable>, Cloneable { - public abstract String getTableName(); + protected String kind; + protected URI uri; + protected String inputSourceId; + protected T startKey; + protected T endKey; + protected long length; + protected ImmutableList hostNames; - @Override - public abstract FragmentProto getProto(); + protected Fragment(String kind, + URI uri, + String inputSourceId, + T startKey, + T endKey, + long length, + String[] hostNames) { + this.kind = kind; + this.uri = uri; + this.inputSourceId = inputSourceId; + this.startKey = startKey; + this.endKey = endKey; + this.length = length; + this.hostNames = hostNames == null ? ImmutableList.of() : ImmutableList.copyOf(hostNames); + } + + /** + * Returns the fragment type. + * + * @return fragment type + */ + public final String getKind() { + return kind; + } + + /** + * Returns an unique URI of the input source. + * + * @return URI of the input source + */ + public final URI getUri() { + return uri; + } - public abstract long getLength(); + /** + * Returns a unique id of the input source. + * + * @return id of the input source + */ + public final String getInputSourceId() { + return this.inputSourceId; + } - public abstract String getKey(); + /** + * Returns the start key of the data range. + * {@link org.apache.tajo.storage.Scanner} will start reading data from the point indicated by this key. + * + * @return start key + */ + public final T getStartKey() { + return startKey; + } - public String[] getHosts(); + /** + * Returns the end key of the data range. + * {@link org.apache.tajo.storage.Scanner} will stop reading data when it reaches the point indicated by this key. + * + * @return end key + */ + public final T getEndKey() { + return endKey; + } - public abstract boolean isEmpty(); + /** + * Returns the length of the data range between start key and end key. + * + * @return length of the range + */ + public final long getLength() { + return length; + } + + /** + * Returns host names which have any portion of the data between start key and end key. + * + * @return host names + */ + public final ImmutableList getHostNames() { + return hostNames; + } + + /** + * Indicates the fragment is empty or not. + * An empty fragment means that there is no data to read. + * + * @return true if the fragment is empty. Otherwise, false. + */ + public boolean isEmpty() { + return length == 0; + } + + /** + * First compares URIs of fragments, and then compares their start keys. + * + * @param t + * @return return 0 if two fragments are same. If not same, return -1 if this fragment is smaller than the other. + * Otherwise, return 1; + */ + @Override + public final int compareTo(Fragment t) { + int cmp = uri.compareTo(t.uri); + if (cmp == 0) { + if (startKey != null && t.startKey != null) { + return startKey.compareTo(t.startKey); + } else if (startKey == null) { // nulls last + return 1; + } else { + return -1; + } + } else { + return cmp; + } + } + + @Override + public Object clone() throws CloneNotSupportedException { + Fragment clone = (Fragment) super.clone(); + clone.uri = this.uri; + clone.inputSourceId = this.inputSourceId; + clone.startKey = this.startKey; + clone.endKey = this.endKey; + clone.hostNames = this.hostNames; + clone.length = this.length; + return clone; + } } diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java index 4ce6928b84..835a714c8a 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentConvertor.java @@ -20,13 +20,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.conf.Configuration; import org.apache.tajo.annotation.ThreadSafe; import org.apache.tajo.exception.TajoInternalError; -import java.io.IOException; -import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.List; import java.util.Map; @@ -34,95 +33,81 @@ @ThreadSafe public class FragmentConvertor { - /** - * Cache of fragment classes - */ - protected static final Map> CACHED_FRAGMENT_CLASSES = Maps.newConcurrentMap(); - /** - * Cache of constructors for each class. - */ - private static final Map, Constructor> CONSTRUCTOR_CACHE = Maps.newConcurrentMap(); - /** - * default parameter for all constructors - */ - private static final Class[] DEFAULT_FRAGMENT_PARAMS = { ByteString.class }; - public static Class getFragmentClass(Configuration conf, String dataFormat) { - Class fragmentClass = CACHED_FRAGMENT_CLASSES.get(dataFormat.toLowerCase()); - if (fragmentClass == null) { - fragmentClass = conf.getClass( - String.format("tajo.storage.fragment.%s.class", dataFormat.toLowerCase()), null, Fragment.class); - CACHED_FRAGMENT_CLASSES.put(dataFormat.toLowerCase(), fragmentClass); + private static final Map SERDE_MAP = Maps.newConcurrentMap(); + + private static FragmentSerde getFragmentSerde(Configuration conf, String fragmentKind) { + fragmentKind = fragmentKind.toLowerCase(); + FragmentSerde serde = SERDE_MAP.get(fragmentKind); + if (serde == null) { + Class serdeClass = conf.getClass( + String.format("tajo.storage.fragment.serde.%s", fragmentKind), null, FragmentSerde.class); + try { + serde = serdeClass.getConstructor(null).newInstance(); + } catch (InstantiationException + | IllegalAccessException + | InvocationTargetException + | NoSuchMethodException e) { + throw new TajoInternalError(e); + } + SERDE_MAP.put(fragmentKind, serde); } - if (fragmentClass == null) { - throw new TajoInternalError("No such a fragment for " + dataFormat.toLowerCase()); + if (serde == null) { + throw new TajoInternalError("No such a serde for " + fragmentKind); } - return fragmentClass; + return serde; } - public static T convert(Class clazz, FragmentProto fragment) { - T result; + public static T convert(Configuration conf, String fragmentKind, FragmentProto fragment) { + FragmentSerde serde = getFragmentSerde(conf, fragmentKind); try { - Constructor constructor = (Constructor) CONSTRUCTOR_CACHE.get(clazz); - if (constructor == null) { - constructor = clazz.getDeclaredConstructor(DEFAULT_FRAGMENT_PARAMS); - constructor.setAccessible(true); - CONSTRUCTOR_CACHE.put(clazz, constructor); - } - result = constructor.newInstance(new Object[]{fragment.getContents()}); - } catch (Throwable e) { + return (T) serde.deserialize( + serde.newBuilder() + .mergeFrom(fragment.getContents()) + .build()); + } catch (InvalidProtocolBufferException e) { throw new TajoInternalError(e); } - - return result; } public static T convert(Configuration conf, FragmentProto fragment) { - Class fragmentClass = (Class) getFragmentClass(conf, fragment.getDataFormat().toLowerCase()); - if (fragmentClass == null) { - throw new TajoInternalError("No such a fragment class for " + fragment.getDataFormat()); - } - return convert(fragmentClass, fragment); + return convert(conf, fragment.getKind(), fragment); } - public static List convert(Class clazz, FragmentProto...fragments) - throws IOException { + public static List convert(Configuration conf, FragmentProto...fragments) { List list = Lists.newArrayList(); if (fragments == null) { return list; } for (FragmentProto proto : fragments) { - list.add(convert(clazz, proto)); + list.add(convert(conf, proto)); } return list; } - public static List convert(Configuration conf, FragmentProto...fragments) { - List list = Lists.newArrayList(); - if (fragments == null) { - return list; - } - for (FragmentProto proto : fragments) { - list.add((T) convert(conf, proto)); - } - return list; + public static FragmentProto toFragmentProto(Configuration conf, Fragment fragment) { + FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder(); + fragmentBuilder.setId(fragment.getInputSourceId()); + fragmentBuilder.setKind(fragment.getKind()); + fragmentBuilder.setContents(getFragmentSerde(conf, fragment.getKind()).serialize(fragment).toByteString()); + return fragmentBuilder.build(); } - public static List toFragmentProtoList(Fragment... fragments) { + public static List toFragmentProtoList(Configuration conf, Fragment... fragments) { List list = Lists.newArrayList(); if (fragments == null) { return list; } for (Fragment fragment : fragments) { - list.add(fragment.getProto()); + list.add(toFragmentProto(conf, fragment)); } return list; } - public static FragmentProto [] toFragmentProtoArray(Fragment... fragments) { - List list = toFragmentProtoList(fragments); + public static FragmentProto [] toFragmentProtoArray(Configuration conf, Fragment... fragments) { + List list = toFragmentProtoList(conf, fragments); return list.toArray(new FragmentProto[list.size()]); } } diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentSerde.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentSerde.java new file mode 100644 index 0000000000..c570c9c020 --- /dev/null +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/fragment/FragmentSerde.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.fragment; + +import com.google.protobuf.Message; +import com.google.protobuf.GeneratedMessage.Builder; + +/** + * FragmentSerde abstracts how a fragment is serialized / deserialized to / from a Protocol Buffer message. + * + * @param Fragment class + * @param

Protocol Buffer Message class corresponding to the Fragment class + */ +public interface FragmentSerde { + + /** + * Creates a new builder of {@link P}. + * + * @return a Protocol Buffer message builder + */ + Builder newBuilder(); + + /** + * Serializes a fragment into a Protocol Buffer message. + * + * @param fragment + * @return a serialized Protocol Buffer message + */ + P serialize(F fragment); + + /** + * Deserializes a Protocol Buffer message to a fragment. + * + * @param proto + * @return a deserialized fragment instance + */ + F deserialize(P proto); +} diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json index 17ac3ba419..1161b9f759 100644 --- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json +++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.json @@ -16,6 +16,10 @@ "handler": "org.apache.tajo.storage.pgsql.PgSQLTablespace", "default-format": "rowstore" }, + "jdbc:mysql": { + "handler": "org.apache.tajo.storage.mysql.MySQLTablespace", + "default-format": "rowstore" + }, "s3": { "handler": "org.apache.tajo.storage.s3.S3TableSpace", "default-format": "text" diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml index 2454714452..ce0ce85393 100644 --- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml +++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml @@ -39,57 +39,41 @@ tajo.storage.scanner-handler - text,json,raw,draw,rcfile,row,parquet,orc,sequencefile,avro,hbase + text,json,raw,draw,rcfile,row,parquet,orc,sequencefile,avro,hbase,ex_http_json - tajo.storage.fragment.text.class + tajo.storage.fragment.kind.file org.apache.tajo.storage.fragment.FileFragment - tajo.storage.fragment.json.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.raw.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.draw.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.rcfile.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.row.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.hbase + org.apache.tajo.storage.hbase.HBaseFragment - tajo.storage.fragment.parquet.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.jdbc + org.apache.tajo.storage.jdbc.JdbcFragment - tajo.storage.fragment.orc.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.example-http + org.apache.tajo.storage.http.ExampleHttpFileFragment - tajo.storage.fragment.sequencefile.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.file + org.apache.tajo.storage.fragment.FileFragmentSerde - tajo.storage.fragment.avro.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.hbase + org.apache.tajo.storage.hbase.HBaseFragmentSerde - tajo.storage.fragment.hbase.class - org.apache.tajo.storage.hbase.HBaseFragment + tajo.storage.fragment.serde.jdbc + org.apache.tajo.storage.jdbc.JdbcFragmentSerde - tajo.storage.fragment.jdbc.class - org.apache.tajo.storage.jdbc.JdbcFragment + tajo.storage.fragment.serde.example-http + org.apache.tajo.storage.http.ExampleHttpFileFragmentSerde @@ -147,7 +131,12 @@ tajo.storage.scanner-handler.hbase.class org.apache.tajo.storage.hbase.HBaseScanner - + + + tajo.storage.scanner-handler.ex_http_json.class + org.apache.tajo.storage.http.ExampleHttpJsonScanner + + tajo.storage.appender-handler diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java index d0ee8e06be..8a25a8c4d9 100644 --- a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java +++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestFrameTuple.java @@ -39,12 +39,11 @@ public void setUp() throws Exception { DatumFactory.createChar('9'), DatumFactory.createInt2((short) 17), DatumFactory.createInt4(59), - DatumFactory.createInt8(23l), + DatumFactory.createInt8(23L), DatumFactory.createFloat4(77.9f), DatumFactory.createFloat8(271.9f), DatumFactory.createText("hyunsik"), DatumFactory.createBlob("hyunsik".getBytes()), - DatumFactory.createInet4("192.168.0.1") }); tuple2 = new VTuple(new Datum[] { @@ -53,12 +52,11 @@ public void setUp() throws Exception { DatumFactory.createChar('9'), DatumFactory.createInt2((short) 17), DatumFactory.createInt4(59), - DatumFactory.createInt8(23l), + DatumFactory.createInt8(23L), DatumFactory.createFloat4(77.9f), DatumFactory.createFloat8(271.9f), DatumFactory.createText("hyunsik"), DatumFactory.createBlob("hyunsik".getBytes()), - DatumFactory.createInet4("192.168.0.1") }); } @@ -69,14 +67,14 @@ public void tearDown() throws Exception { @Test public final void testFrameTuple() { Tuple frame = new FrameTuple(tuple1, tuple2); - assertEquals(22, frame.size()); - for (int i = 0; i < 22; i++) { + assertEquals(20, frame.size()); + for (int i = 0; i < frame.size(); i++) { assertTrue(frame.contains(i)); } - assertEquals(23l, frame.getInt8(5)); - assertEquals(23l, frame.getInt8(16)); - assertEquals("192.168.0.1", frame.getText(10)); - assertEquals("192.168.0.1", frame.getText(21)); + assertEquals(23L, frame.getInt8(5)); + assertEquals(23L, frame.getInt8(15)); + assertEquals("hyunsik", frame.getText(8)); + assertEquals("hyunsik", frame.getText(19)); } } diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java index 76c2ef153d..29c79b329a 100644 --- a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java +++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java @@ -52,9 +52,8 @@ public void setUp() { .add("col7", TajoDataTypes.Type.FLOAT8) .add("col8", TajoDataTypes.Type.TEXT) .add("col9", TajoDataTypes.Type.BLOB) - .add("col10", TajoDataTypes.Type.INET4) - .add("col11", TajoDataTypes.Type.INT4) - .add("col12", TajoDataTypes.Type.NULL_TYPE) + .add("col10", TajoDataTypes.Type.INT4) + .add("col11", TajoDataTypes.Type.NULL_TYPE) .build(); StringBuilder sb = new StringBuilder(); @@ -67,7 +66,6 @@ public void setUp() { sb.append(DatumFactory.createFloat8(271.9f)).append('|'); sb.append(DatumFactory.createText("str2")).append('|'); sb.append(DatumFactory.createBlob("jinho")).append('|'); - sb.append(DatumFactory.createInet4("192.168.0.1")).append('|'); sb.append(new String(nullbytes)).append('|'); sb.append(NullDatum.get()); textRow = BytesUtils.splitPreserveAllTokens(sb.toString().getBytes(), '|', 13); @@ -88,9 +86,8 @@ public void testGetDatum() { assertEquals(DatumFactory.createFloat8(271.9f), t1.get(6)); assertEquals(DatumFactory.createText("str2"), t1.get(7)); assertEquals(DatumFactory.createBlob("jinho".getBytes()), t1.get(8)); - assertEquals(DatumFactory.createInet4("192.168.0.1"), t1.get(9)); + assertEquals(NullDatum.get(), t1.get(9)); assertEquals(NullDatum.get(), t1.get(10)); - assertEquals(NullDatum.get(), t1.get(11)); } @Test @@ -113,7 +110,6 @@ public void testContain() { assertFalse(t1.contains(8)); assertFalse(t1.contains(9)); assertFalse(t1.contains(10)); - assertFalse(t1.contains(11)); } @Test @@ -122,14 +118,14 @@ public void testPut() { LazyTuple t1 = new LazyTuple(schema, new byte[colNum][], -1); t1.put(0, DatumFactory.createText("str")); t1.put(1, DatumFactory.createInt4(2)); - t1.put(11, DatumFactory.createFloat4(0.76f)); + t1.put(10, DatumFactory.createFloat4(0.76f)); assertTrue(t1.contains(0)); assertTrue(t1.contains(1)); assertEquals(t1.getText(0), "str"); assertEquals(t1.get(1).asInt4(), 2); - assertTrue(t1.get(11).asFloat4() == 0.76f); + assertTrue(t1.get(10).asFloat4() == 0.76f); } @Test diff --git a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml index 1c4530a3cd..1737e221c7 100644 --- a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml +++ b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml @@ -38,57 +38,33 @@ tajo.storage.scanner-handler - text,json,raw,draw,rcfile,row,parquet,orc,sequencefile,avro,hbase + text,json,raw,draw,rcfile,row,parquet,orc,sequencefile,avro,hbase,ex_http_json - tajo.storage.fragment.text.class + tajo.storage.fragment.kind.file org.apache.tajo.storage.fragment.FileFragment - tajo.storage.fragment.json.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.raw.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.draw.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.rcfile.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.row.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.parquet.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.orc.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.hbase + org.apache.tajo.storage.hbase.HBaseFragment - tajo.storage.fragment.sequencefile.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.jdbc + org.apache.tajo.storage.jdbc.JdbcFragment - tajo.storage.fragment.avro.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.file + org.apache.tajo.storage.fragment.FileFragmentSerde - tajo.storage.fragment.hbase.class - org.apache.tajo.storage.hbase.HBaseFragment + tajo.storage.fragment.serde.hbase + org.apache.tajo.storage.hbase.HBaseFragmentSerde - tajo.storage.fragment.jdbc.class - org.apache.tajo.storage.jdbc.JdbcFragment + tajo.storage.fragment.serde.jdbc + org.apache.tajo.storage.jdbc.JdbcFragmentSerde @@ -150,6 +126,11 @@ org.apache.tajo.storage.hbase.HBaseScanner + + tajo.storage.scanner-handler.ex_http_json.class + org.apache.tajo.storage.http.ExampleHttpJsonScanner + + tajo.storage.appender-handler diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java index 18aa515b93..e1026bb254 100644 --- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java +++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragment.java @@ -19,112 +19,49 @@ package org.apache.tajo.storage.hbase; import com.google.common.base.Objects; -import com.google.gson.annotations.Expose; -import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.tajo.BuiltinStorages; -import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; +import org.apache.tajo.TajoConstants; +import org.apache.tajo.storage.fragment.BuiltinFragmentKinds; import org.apache.tajo.storage.fragment.Fragment; -import org.apache.tajo.storage.hbase.StorageFragmentProtos.HBaseFragmentProto; +import org.apache.tajo.storage.hbase.HBaseFragment.HBaseFragmentKey; import java.net.URI; -public class HBaseFragment implements Fragment, Comparable, Cloneable { - @Expose - private URI uri; - @Expose - private String tableName; - @Expose +/** + * Fragment for HBase + */ +public class HBaseFragment extends Fragment { private String hbaseTableName; - @Expose - private byte[] startRow; - @Expose - private byte[] stopRow; - @Expose - private String regionLocation; - @Expose private boolean last; - @Expose - private long length; public HBaseFragment(URI uri, String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, String regionLocation) { - this.uri = uri; - this.tableName = tableName; + super(BuiltinFragmentKinds.HBASE, uri, tableName, new HBaseFragmentKey(startRow), new HBaseFragmentKey(stopRow), + TajoConstants.UNKNOWN_LENGTH, new String[]{regionLocation}); + this.hbaseTableName = hbaseTableName; - this.startRow = startRow; - this.stopRow = stopRow; - this.regionLocation = regionLocation; this.last = false; } - public HBaseFragment(ByteString raw) throws InvalidProtocolBufferException { - HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder(); - builder.mergeFrom(raw); - builder.build(); - init(builder.build()); - } - - private void init(HBaseFragmentProto proto) { - this.uri = URI.create(proto.getUri()); - this.tableName = proto.getTableName(); - this.hbaseTableName = proto.getHbaseTableName(); - this.startRow = proto.getStartRow().toByteArray(); - this.stopRow = proto.getStopRow().toByteArray(); - this.regionLocation = proto.getRegionLocation(); - this.length = proto.getLength(); - this.last = proto.getLast(); - } - - @Override - public int compareTo(HBaseFragment t) { - return Bytes.compareTo(startRow, t.startRow); - } - - public URI getUri() { - return uri; - } - - @Override - public String getTableName() { - return tableName; - } - - @Override - public String getKey() { - return new String(startRow); + public HBaseFragment(URI uri, String tableName, String hbaseTableName, byte[] startRow, byte[] stopRow, + String regionLocation, boolean last) { + this(uri, tableName, hbaseTableName, startRow, stopRow, regionLocation); + this.last = last; } @Override public boolean isEmpty() { - return startRow == null || stopRow == null; - } - - @Override - public long getLength() { - return length; + return startKey.isEmpty() || endKey.isEmpty(); } public void setLength(long length) { this.length = length; } - @Override - public String[] getHosts() { - return new String[] {regionLocation}; - } - public Object clone() throws CloneNotSupportedException { HBaseFragment frag = (HBaseFragment) super.clone(); - frag.uri = uri; - frag.tableName = tableName; frag.hbaseTableName = hbaseTableName; - frag.startRow = startRow; - frag.stopRow = stopRow; - frag.regionLocation = regionLocation; frag.last = last; - frag.length = length; return frag; } @@ -132,9 +69,9 @@ public Object clone() throws CloneNotSupportedException { public boolean equals(Object o) { if (o instanceof HBaseFragment) { HBaseFragment t = (HBaseFragment) o; - if (tableName.equals(t.tableName) - && Bytes.equals(startRow, t.startRow) - && Bytes.equals(stopRow, t.stopRow)) { + if (inputSourceId.equals(t.inputSourceId) + && startKey.equals(t.startKey) + && endKey.equals(t.endKey)) { return true; } } @@ -143,51 +80,19 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hashCode(tableName, hbaseTableName, startRow, stopRow); + return Objects.hashCode(inputSourceId, hbaseTableName, startKey, endKey); } @Override public String toString() { return - "\"fragment\": {\"uri:\"" + uri.toString() +"\", \"tableName\": \""+ tableName + + "\"fragment\": {\"uri:\"" + uri.toString() +"\", \"tableName\": \""+ inputSourceId + "\", hbaseTableName\": \"" + hbaseTableName + "\"" + - ", \"startRow\": \"" + new String(startRow) + "\"" + - ", \"stopRow\": \"" + new String(stopRow) + "\"" + + ", \"startRow\": \"" + new String(startKey.bytes) + "\"" + + ", \"stopRow\": \"" + new String(endKey.bytes) + "\"" + ", \"length\": \"" + length + "\"}" ; } - @Override - public FragmentProto getProto() { - HBaseFragmentProto.Builder builder = HBaseFragmentProto.newBuilder(); - builder - .setUri(uri.toString()) - .setTableName(tableName) - .setHbaseTableName(hbaseTableName) - .setStartRow(ByteString.copyFrom(startRow)) - .setStopRow(ByteString.copyFrom(stopRow)) - .setLast(last) - .setLength(length) - .setRegionLocation(regionLocation); - - FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder(); - fragmentBuilder.setId(this.tableName); - fragmentBuilder.setContents(builder.buildPartial().toByteString()); - fragmentBuilder.setDataFormat(BuiltinStorages.HBASE); - return fragmentBuilder.build(); - } - - public byte[] getStartRow() { - return startRow; - } - - public byte[] getStopRow() { - return stopRow; - } - - public String getRegionLocation() { - return regionLocation; - } - public boolean isLast() { return last; } @@ -200,15 +105,51 @@ public String getHbaseTableName() { return hbaseTableName; } - public void setHbaseTableName(String hbaseTableName) { - this.hbaseTableName = hbaseTableName; - } - public void setStartRow(byte[] startRow) { - this.startRow = startRow; + this.startKey = new HBaseFragmentKey(startRow); } public void setStopRow(byte[] stopRow) { - this.stopRow = stopRow; + this.endKey = new HBaseFragmentKey(stopRow); + } + + public static class HBaseFragmentKey implements Comparable { + private final byte[] bytes; + + public HBaseFragmentKey(byte[] key) { + this.bytes = key; + } + + public byte[] getBytes() { + return bytes; + } + + @Override + public int hashCode() { + return Bytes.hashCode(bytes); + } + + @Override + public boolean equals(Object o) { + if (o instanceof HBaseFragmentKey) { + HBaseFragmentKey other = (HBaseFragmentKey) o; + return Bytes.equals(bytes, other.bytes); + } + return false; + } + + @Override + public int compareTo(HBaseFragmentKey o) { + return Bytes.compareTo(bytes, o.bytes); + } + + @Override + public String toString() { + return new String(bytes); + } + + public boolean isEmpty() { + return this.bytes == null; + } } } diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragmentSerde.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragmentSerde.java new file mode 100644 index 0000000000..f896f43519 --- /dev/null +++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseFragmentSerde.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.hbase; + +import com.google.protobuf.ByteString; +import com.google.protobuf.GeneratedMessage.Builder; +import org.apache.tajo.storage.fragment.FragmentSerde; +import org.apache.tajo.storage.hbase.StorageFragmentProtos.HBaseFragmentProto; + +import java.net.URI; + +public class HBaseFragmentSerde implements FragmentSerde { + + @Override + public Builder newBuilder() { + return HBaseFragmentProto.newBuilder(); + } + + @Override + public HBaseFragmentProto serialize(HBaseFragment fragment) { + return HBaseFragmentProto.newBuilder() + .setUri(fragment.getUri().toASCIIString()) + .setTableName(fragment.getInputSourceId()) + .setHbaseTableName(fragment.getHbaseTableName()) + .setStartRow(ByteString.copyFrom(fragment.getStartKey().getBytes())) + .setStopRow(ByteString.copyFrom(fragment.getEndKey().getBytes())) + .setLast(fragment.isLast()) + .setLength(fragment.getLength()) + .setRegionLocation(fragment.getHostNames().get(0)) + .build(); + } + + @Override + public HBaseFragment deserialize(HBaseFragmentProto proto) { + return new HBaseFragment( + URI.create(proto.getUri()), + proto.getTableName(), + proto.getHbaseTableName(), + proto.getStartRow().toByteArray(), + proto.getStopRow().toByteArray(), + proto.getRegionLocation(), + proto.getLast()); + } +} diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java index b2ca02dffc..781f911b4f 100644 --- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java +++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseScanner.java @@ -36,10 +36,16 @@ import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.NullDatum; import org.apache.tajo.datum.TextDatum; -import org.apache.tajo.exception.*; +import org.apache.tajo.exception.TajoException; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.storage.*; +import org.apache.tajo.storage.Scanner; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.util.BytesUtils; @@ -175,16 +181,16 @@ private void initScanner() throws IOException { } } - scan.setStartRow(fragment.getStartRow()); - if (fragment.isLast() && fragment.getStopRow() != null && - fragment.getStopRow().length > 0) { + scan.setStartRow(fragment.getStartKey().getBytes()); + if (fragment.isLast() && !fragment.getEndKey().isEmpty() && + fragment.getEndKey().getBytes().length > 0) { // last and stopRow is not empty if (filters == null) { filters = new FilterList(); } - filters.addFilter(new InclusiveStopFilter(fragment.getStopRow())); + filters.addFilter(new InclusiveStopFilter(fragment.getEndKey().getBytes())); } else { - scan.setStopRow(fragment.getStopRow()); + scan.setStopRow(fragment.getEndKey().getBytes()); } if (filters != null) { diff --git a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java index 132ceff0ae..2dbd137682 100644 --- a/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java +++ b/tajo-storage/tajo-storage-hbase/src/main/java/org/apache/tajo/storage/hbase/HBaseTablespace.java @@ -396,7 +396,7 @@ public void purgeTable(TableDesc tableDesc) throws IOException, TajoException { } @Override - public URI getTableUri(String databaseName, String tableName) { + public URI getTableUri(TableMeta meta, String databaseName, String tableName) { return URI.create(uri.toString() + "/" + tableName); } @@ -525,10 +525,10 @@ private Collection convertRangeToFragment( if (fragmentMap.containsKey(regionStartKey)) { final HBaseFragment prevFragment = fragmentMap.get(regionStartKey); - if (Bytes.compareTo(fragmentStart, prevFragment.getStartRow()) < 0) { + if (Bytes.compareTo(fragmentStart, prevFragment.getStartKey().getBytes()) < 0) { prevFragment.setStartRow(fragmentStart); } - if (Bytes.compareTo(fragmentStop, prevFragment.getStopRow()) > 0) { + if (Bytes.compareTo(fragmentStop, prevFragment.getEndKey().getBytes()) > 0) { prevFragment.setStopRow(fragmentStop); } } else { @@ -557,6 +557,7 @@ private Collection convertRangeToFragment( @Override public List getSplits(String inputSourceId, TableDesc table, + boolean requireSorted, @Nullable EvalNode filterCondition) throws IOException, TajoException { return (List) (List) getRawSplits(inputSourceId, table, filterCondition); } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java index 94e07e5193..58001b934c 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileScanner.java @@ -18,8 +18,6 @@ package org.apache.tajo.storage; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,33 +30,32 @@ import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.plan.logical.LogicalNode; -import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.storage.fragment.AbstractFileFragment; import org.apache.tajo.storage.fragment.Fragment; import java.io.IOException; public abstract class FileScanner implements Scanner { - private static final Log LOG = LogFactory.getLog(FileScanner.class); protected boolean inited = false; protected final Configuration conf; protected final TableMeta meta; protected final Schema schema; - protected final FileFragment fragment; + protected final AbstractFileFragment fragment; protected final int columnNum; protected Column [] targets; protected float progress; - protected TableStats tableStats; + protected TableStats inputStats; public FileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) { this.conf = conf; this.meta = meta; this.schema = schema; - this.fragment = (FileFragment)fragment; - this.tableStats = new TableStats(); + this.fragment = (AbstractFileFragment)fragment; + this.inputStats = new TableStats(); this.columnNum = this.schema.size(); } @@ -67,14 +64,14 @@ public void init() throws IOException { progress = 0.0f; if (fragment != null) { - tableStats.setNumBytes(fragment.getLength()); - tableStats.setNumBlocks(1); + inputStats.setNumBytes(fragment.getLength()); + inputStats.setNumBlocks(1); } if (schema != null) { for(Column eachColumn: schema.getRootColumns()) { ColumnStats columnStats = new ColumnStats(eachColumn); - tableStats.addColumnStat(columnStats); + inputStats.addColumnStat(columnStats); } } } @@ -112,6 +109,6 @@ public float getProgress() { @Override public TableStats getInputStats() { - return tableStats; + return inputStats; } } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java index ad4574909d..2785de4ea4 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java @@ -40,6 +40,7 @@ import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.logical.LogicalNode; import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.storage.fragment.Fragment; import org.apache.tajo.util.Bytes; @@ -125,7 +126,7 @@ protected void storageInit() throws IOException { } @Override - public long getTableVolume(TableDesc table, Optional filter) throws UnsupportedException { + public long getTableVolume(TableDesc table, Optional filter) { Path path = new Path(table.getUri()); ContentSummary summary; try { @@ -168,7 +169,7 @@ public boolean exists(Path path) throws IOException { } @Override - public URI getTableUri(String databaseName, String tableName) { + public URI getTableUri(TableMeta meta, String databaseName, String tableName) { return StorageUtil.concatPath(spacePath, databaseName, tableName).toUri(); } @@ -303,10 +304,12 @@ public boolean accept(Path path) { * Subclasses may override to, e.g., select only files matching a regular * expression. * + * @param requireSort if set, result will be sorted by their paths. + * @param dirs input dirs * @return array of FileStatus objects * @throws IOException if zero items. */ - protected List listStatus(Path... dirs) throws IOException { + protected List listStatus(boolean requireSort, Path... dirs) throws IOException { List result = new ArrayList<>(); if (dirs.length == 0) { throw new IOException("No input paths specified in job"); @@ -341,6 +344,10 @@ protected List listStatus(Path... dirs) throws IOException { if (!errors.isEmpty()) { throw new InvalidInputException(errors); } + + if (requireSort) { + Collections.sort(result); + } LOG.info("Total input paths to process : " + result.size()); return result; } @@ -446,8 +453,8 @@ public long getMinSplitSize() { /** * Get Disk Ids by Volume Bytes */ - private int[] getDiskIds(VolumeId[] volumeIds) { - int[] diskIds = new int[volumeIds.length]; + private Integer[] getDiskIds(VolumeId[] volumeIds) { + Integer[] diskIds = new Integer[volumeIds.length]; for (int i = 0; i < volumeIds.length; i++) { int diskId = -1; if (volumeIds[i] != null && volumeIds[i].hashCode() > 0) { @@ -463,7 +470,7 @@ private int[] getDiskIds(VolumeId[] volumeIds) { * * @throws IOException */ - public List getSplits(String tableName, TableMeta meta, Schema schema, Path... inputs) + public List getSplits(String tableName, TableMeta meta, Schema schema, boolean requireSort, Path... inputs) throws IOException { // generate splits' @@ -476,7 +483,7 @@ public List getSplits(String tableName, TableMeta meta, Schema schema, if (fs.isFile(p)) { files.addAll(Lists.newArrayList(fs.getFileStatus(p))); } else { - files.addAll(listStatus(p)); + files.addAll(listStatus(requireSort, p)); } int previousSplitSize = splits.size(); @@ -605,14 +612,15 @@ public String getMessage(){ @Override public List getSplits(String inputSourceId, TableDesc table, + boolean requireSort, @Nullable EvalNode filterCondition) throws IOException { - return getSplits(inputSourceId, table.getMeta(), table.getSchema(), new Path(table.getUri())); + return getSplits(inputSourceId, table.getMeta(), table.getSchema(), requireSort, new Path(table.getUri())); } @Override public void createTable(TableDesc tableDesc, boolean ifNotExists) throws IOException { if (!tableDesc.isExternal()) { - String [] splitted = CatalogUtil.splitFQTableName(tableDesc.getName()); + String [] splitted = IdentifierUtil.splitFQTableName(tableDesc.getName()); String databaseName = splitted[0]; String simpleTableName = splitted[1]; diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java index f31b85cef8..348bb3cd7c 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RawFile.java @@ -92,7 +92,7 @@ public void init() throws IOException { fis = new FileInputStream(file); channel = fis.getChannel(); filePosition = startOffset = fragment.getStartKey(); - endOffset = fragment.getStartKey() + fragment.getLength(); + endOffset = fragment.getEndKey(); if (LOG.isDebugEnabled()) { LOG.debug("RawFileScanner open:" + fragment + "," + channel.position() + ", file size :" + channel.size() @@ -343,10 +343,6 @@ public Tuple next() throws IOException { break; } - case INET4: - outTuple.put(i, DatumFactory.createInet4(buffer.getInt())); - break; - case DATE: { int val = buffer.getInt(); if (val < Integer.MIN_VALUE + 1) { @@ -439,12 +435,12 @@ public boolean isSplittable(){ @Override public TableStats getInputStats() { - if(tableStats != null){ - tableStats.setNumRows(recordCount); - tableStats.setReadBytes(totalReadBytes); // actual read bytes (scan + rescan * n) - tableStats.setNumBytes(fragment.getLength()); + if(inputStats != null){ + inputStats.setNumRows(recordCount); + inputStats.setReadBytes(totalReadBytes); // actual read bytes (scan + rescan * n) + inputStats.setNumBytes(fragment.getLength()); } - return tableStats; + return inputStats; } @Override diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java deleted file mode 100644 index a7c6b7146c..0000000000 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/RowFile.java +++ /dev/null @@ -1,515 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.storage; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IOUtils; -import org.apache.tajo.TaskAttemptId; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.TableMeta; -import org.apache.tajo.catalog.statistics.TableStats; -import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.exception.TajoRuntimeException; -import org.apache.tajo.exception.UnsupportedException; -import org.apache.tajo.plan.expr.EvalNode; -import org.apache.tajo.plan.serder.PlanProto.ShuffleType; -import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.exception.AlreadyExistsStorageException; -import org.apache.tajo.storage.fragment.Fragment; -import org.apache.tajo.util.BitArray; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.Arrays; - -public class RowFile { - public static final Log LOG = LogFactory.getLog(RowFile.class); - - private static final int SYNC_ESCAPE = -1; - private static final int SYNC_HASH_SIZE = 16; - private static final int SYNC_SIZE = 4 + SYNC_HASH_SIZE; - private final static int DEFAULT_BUFFER_SIZE = 65535; - public static int SYNC_INTERVAL; - - public static class RowFileScanner extends FileScanner { - private FileSystem fs; - private FSDataInputStream in; - private Tuple tuple; - - private byte[] sync = new byte[SYNC_HASH_SIZE]; - private byte[] checkSync = new byte[SYNC_HASH_SIZE]; - private long start, end; - - private ByteBuffer buffer; - private final int tupleHeaderSize; - private BitArray nullFlags; - private long bufferStartPos; - - public RowFileScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) - throws IOException { - super(conf, schema, meta, fragment); - - SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname, - ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal) * SYNC_SIZE; - - nullFlags = new BitArray(schema.size()); - tupleHeaderSize = nullFlags.bytesLength() + (2 * Short.SIZE / 8); - this.start = this.fragment.getStartKey(); - this.end = this.start + this.fragment.getLength(); - } - - public void init() throws IOException { - // set default page size. - fs = fragment.getPath().getFileSystem(conf); - in = fs.open(fragment.getPath()); - buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE * schema.size()); - buffer.flip(); - - readHeader(); - - // find the correct position from the start - if (this.start > in.getPos()) { - long realStart = start > SYNC_SIZE ? (start-SYNC_SIZE) : 0; - in.seek(realStart); - } - bufferStartPos = in.getPos(); - fillBuffer(); - - if (start != 0) { - // TODO: improve - boolean syncFound = false; - while (!syncFound) { - if (buffer.remaining() < SYNC_SIZE) { - fillBuffer(); - } - buffer.mark(); - syncFound = checkSync(); - if (!syncFound) { - buffer.reset(); - buffer.get(); // proceed one byte - } - } - bufferStartPos += buffer.position(); - buffer.compact(); - buffer.flip(); - } - - tuple = new VTuple(schema.size()); - - super.init(); - } - - private void readHeader() throws IOException { - SYNC_INTERVAL = in.readInt(); - StorageUtil.readFully(in, this.sync, 0, SYNC_HASH_SIZE); - } - - /** - * Find the sync from the front of the buffer - * - * @return return true if it succeeds to find the sync. - * @throws java.io.IOException - */ - private boolean checkSync() throws IOException { - buffer.getInt(); // escape - buffer.get(checkSync, 0, SYNC_HASH_SIZE); // sync - return Arrays.equals(checkSync, sync); - } - - private int fillBuffer() throws IOException { - bufferStartPos += buffer.position(); - buffer.compact(); - int remain = buffer.remaining(); - int read = in.read(buffer); - if (read == -1) { - buffer.flip(); - return read; - } else { - int totalRead = read; - if (remain > totalRead) { - read = in.read(buffer); - totalRead += read > 0 ? read : 0; - } - buffer.flip(); - return totalRead; - } - } - - @Override - public Tuple next() throws IOException { - while (buffer.remaining() < SYNC_SIZE) { - if (fillBuffer() < 0) { - return null; - } - } - - buffer.mark(); - if (!checkSync()) { - buffer.reset(); - } else { - if (bufferStartPos + buffer.position() > end) { - return null; - } - } - - while (buffer.remaining() < tupleHeaderSize) { - if (fillBuffer() < 0) { - return null; - } - } - - int i; - - int nullFlagSize = buffer.getShort(); - byte[] nullFlagBytes = new byte[nullFlagSize]; - buffer.get(nullFlagBytes, 0, nullFlagSize); - nullFlags = new BitArray(nullFlagBytes); - int tupleSize = buffer.getShort(); - - while (buffer.remaining() < (tupleSize)) { - if (fillBuffer() < 0) { - return null; - } - } - - Datum datum; - Column col; - for (i = 0; i < schema.size(); i++) { - if (!nullFlags.get(i)) { - col = schema.getColumn(i); - switch (col.getDataType().getType()) { - case BOOLEAN : - datum = DatumFactory.createBool(buffer.get()); - tuple.put(i, datum); - break; - - case BIT: - datum = DatumFactory.createBit(buffer.get()); - tuple.put(i, datum ); - break; - - case CHAR : - int realLen = buffer.getInt(); - byte[] buf = new byte[col.getDataType().getLength()]; - buffer.get(buf); - byte[] charBuf = Arrays.copyOf(buf, realLen); - tuple.put(i, DatumFactory.createChar(charBuf)); - break; - - case INT2 : - datum = DatumFactory.createInt2(buffer.getShort()); - tuple.put(i, datum ); - break; - - case INT4 : - datum = DatumFactory.createInt4(buffer.getInt()); - tuple.put(i, datum ); - break; - - case INT8 : - datum = DatumFactory.createInt8(buffer.getLong()); - tuple.put(i, datum ); - break; - - case FLOAT4 : - datum = DatumFactory.createFloat4(buffer.getFloat()); - tuple.put(i, datum); - break; - - case FLOAT8 : - datum = DatumFactory.createFloat8(buffer.getDouble()); - tuple.put(i, datum); - break; - - case TEXT: - short bytelen = buffer.getShort(); - byte[] strbytes = new byte[bytelen]; - buffer.get(strbytes, 0, bytelen); - datum = DatumFactory.createText(strbytes); - tuple.put(i, datum); - break; - - case BLOB: - short bytesLen = buffer.getShort(); - byte [] bytesBuf = new byte[bytesLen]; - buffer.get(bytesBuf); - datum = DatumFactory.createBlob(bytesBuf); - tuple.put(i, datum); - break; - - case INET4 : - byte[] ipv4 = new byte[4]; - buffer.get(ipv4, 0, 4); - datum = DatumFactory.createInet4(ipv4); - tuple.put(i, datum); - break; - - default: - break; - } - } else { - tuple.put(i, DatumFactory.createNullDatum()); - } - } - return tuple; - } - - @Override - public void reset() throws IOException { - init(); - } - - @Override - public void close() throws IOException { - if (in != null) { - in.close(); - } - } - - @Override - public boolean isProjectable() { - return false; - } - - @Override - public boolean isSelectable() { - return false; - } - - @Override - public void setFilter(EvalNode filter) { - throw new TajoRuntimeException(new UnsupportedException()); - } - - @Override - public boolean isSplittable(){ - return true; - } - } - - public static class RowFileAppender extends FileAppender { - private FSDataOutputStream out; - private long lastSyncPos; - private FileSystem fs; - private byte[] sync; - private ByteBuffer buffer; - - private BitArray nullFlags; - // statistics - private TableStatistics stats; - private ShuffleType shuffleType; - - public RowFileAppender(Configuration conf, final TaskAttemptId taskAttemptId, - final Schema schema, final TableMeta meta, final Path workDir) - throws IOException { - super(conf, taskAttemptId, schema, meta, workDir); - } - - public void init() throws IOException { - SYNC_INTERVAL = conf.getInt(ConfVars.ROWFILE_SYNC_INTERVAL.varname, - ConfVars.ROWFILE_SYNC_INTERVAL.defaultIntVal); - fs = path.getFileSystem(conf); - - if (!fs.exists(path.getParent())) { - throw new FileNotFoundException(path.toString()); - } - - if (fs.exists(path)) { - throw new AlreadyExistsStorageException(path); - } - - sync = new byte[SYNC_HASH_SIZE]; - lastSyncPos = 0; - - out = fs.create(path); - - MessageDigest md; - try { - md = MessageDigest.getInstance("MD5"); - md.update((path.toString()+System.currentTimeMillis()).getBytes()); - sync = md.digest(); - } catch (NoSuchAlgorithmException e) { - LOG.error(e); - } - - writeHeader(); - - buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE); - - nullFlags = new BitArray(schema.size()); - - if (tableStatsEnabled) { - this.stats = new TableStatistics(this.schema, columnStatsEnabled); - this.shuffleType = PlannerUtil.getShuffleType( - meta.getProperty(StorageConstants.SHUFFLE_TYPE, - PlannerUtil.getShuffleType(ShuffleType.NONE_SHUFFLE))); - } - } - - private void writeHeader() throws IOException { - out.writeInt(SYNC_INTERVAL); - out.write(sync); - out.flush(); - lastSyncPos = out.getPos(); - } - - @Override - public void addTuple(Tuple t) throws IOException { - checkAndWriteSync(); - Column col; - - buffer.clear(); - nullFlags.clear(); - - for (int i = 0; i < schema.size(); i++) { - if (shuffleType == ShuffleType.RANGE_SHUFFLE) { - // it is to calculate min/max values, and it is only used for the intermediate file. - stats.analyzeField(i, t); - } - - if (t.isBlankOrNull(i)) { - nullFlags.set(i); - } else { - col = schema.getColumn(i); - switch (col.getDataType().getType()) { - case BOOLEAN: - buffer.put(t.getByte(i)); - break; - case BIT: - buffer.put(t.getByte(i)); - break; - case CHAR: - byte[] src = t.getBytes(i); - byte[] dst = Arrays.copyOf(src, col.getDataType().getLength()); - buffer.putInt(src.length); - buffer.put(dst); - break; - case TEXT: - byte [] strbytes = t.getBytes(i); - buffer.putShort((short)strbytes.length); - buffer.put(strbytes, 0, strbytes.length); - break; - case INT2: - buffer.putShort(t.getInt2(i)); - break; - case INT4: - buffer.putInt(t.getInt4(i)); - break; - case INT8: - buffer.putLong(t.getInt8(i)); - break; - case FLOAT4: - buffer.putFloat(t.getFloat4(i)); - break; - case FLOAT8: - buffer.putDouble(t.getFloat8(i)); - break; - case BLOB: - byte [] bytes = t.getBytes(i); - buffer.putShort((short)bytes.length); - buffer.put(bytes); - break; - case INET4: - buffer.put(t.getBytes(i)); - break; - case INET6: - buffer.put(t.getBytes(i)); - break; - case NULL_TYPE: - nullFlags.set(i); - break; - default: - break; - } - } - } - - byte[] bytes = nullFlags.toArray(); - out.writeShort(bytes.length); - out.write(bytes); - - bytes = buffer.array(); - int dataLen = buffer.position(); - out.writeShort(dataLen); - out.write(bytes, 0, dataLen); - - // Statistical section - if (tableStatsEnabled) { - stats.incrementRow(); - } - } - - @Override - public long getOffset() throws IOException { - return out.getPos(); - } - - @Override - public void flush() throws IOException { - out.flush(); - } - - @Override - public void close() throws IOException { - if (out != null) { - if (tableStatsEnabled) { - stats.setNumBytes(out.getPos()); - } - sync(); - out.flush(); - IOUtils.cleanup(LOG, out); - } - } - - private void sync() throws IOException { - if (lastSyncPos != out.getPos()) { - out.writeInt(SYNC_ESCAPE); - out.write(sync); - lastSyncPos = out.getPos(); - } - } - - private void checkAndWriteSync() throws IOException { - if (out.getPos() >= lastSyncPos + SYNC_INTERVAL) { - sync(); - } - } - - @Override - public TableStats getStats() { - if (tableStatsEnabled) { - return stats.getTableStat(); - } else { - return null; - } - } - } -} diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java index ad48850565..29dbcf5b3f 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/avro/AvroScanner.java @@ -51,7 +51,6 @@ * FileScanner for reading Avro files */ public class AvroScanner extends FileScanner { - private Schema avroSchema; private List avroFields; private DataFileReader dataFileReader; private int[] projectionMap; @@ -82,7 +81,7 @@ public void init() throws IOException { prepareProjection(targets); outTuple = new VTuple(projectionMap.length); - avroSchema = AvroUtil.getAvroSchema(meta, conf); + Schema avroSchema = AvroUtil.getAvroSchema(meta, conf); avroFields = avroSchema.getFields(); DatumReader datumReader = new GenericDatumReader<>(avroSchema); @@ -141,15 +140,13 @@ private Datum convertBytes(Object value, TajoDataTypes.Type tajoType, byte[] bytes = new byte[buffer.capacity()]; buffer.get(bytes, 0, bytes.length); switch (tajoType) { - case INET4: - return DatumFactory.createInet4(bytes); case PROTOBUF: try { ProtobufDatumFactory factory = ProtobufDatumFactory.get(dataType.getCode()); Message.Builder builder = factory.newBuilder(); builder.mergeFrom(bytes); - return factory.createDatum(builder); + return ProtobufDatumFactory.createDatum(builder); } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/AbstractFileFragment.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/AbstractFileFragment.java new file mode 100644 index 0000000000..cf726e2bc7 --- /dev/null +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/AbstractFileFragment.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.fragment; + +import org.apache.hadoop.fs.Path; + +import java.net.URI; + +/** + * Abstract fragment implementation for file systems. + */ +public abstract class AbstractFileFragment extends Fragment { + + protected AbstractFileFragment(String kind, + URI uri, + String inputSourceId, + long startKey, + long endKey, + long length, + String[] hostNames) { + super(kind, uri, inputSourceId, startKey, endKey, length, hostNames); + } + + public Path getPath() { + return new Path(uri); + } +} diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java index 8f18c7a0f0..0ead6009a4 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragment.java @@ -19,161 +19,61 @@ package org.apache.tajo.storage.fragment; import com.google.common.base.Objects; -import com.google.gson.annotations.Expose; -import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.Path; -import org.apache.tajo.BuiltinStorages; -import org.apache.tajo.storage.StorageFragmentProtos.FileFragmentProto; +import org.apache.tajo.storage.DataLocation; import org.apache.tajo.util.TUtil; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; -import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto; - -public class FileFragment implements Fragment, Comparable, Cloneable { - @Expose private String tableName; // required - @Expose private Path uri; // required - @Expose public Long startOffset; // required - @Expose public Long length; // required - - private String[] hosts; // Datanode hostnames - @Expose private int[] diskIds; - - public FileFragment(ByteString raw) throws InvalidProtocolBufferException { - FileFragmentProto.Builder builder = FileFragmentProto.newBuilder(); - builder.mergeFrom(raw); - builder.build(); - init(builder.build()); - } +/** + * Fragment for file systems. + */ +public class FileFragment extends AbstractFileFragment { + private Integer[] diskIds; // disk volume ids public FileFragment(String tableName, Path uri, BlockLocation blockLocation) throws IOException { - this.set(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(), blockLocation.getHosts(), null); + this(tableName, uri, blockLocation.getOffset(), blockLocation.getLength(), blockLocation.getHosts(), null); } - public FileFragment(String tableName, Path uri, long start, long length, String[] hosts, int[] diskIds) { - this.set(tableName, uri, start, length, hosts, diskIds); + public FileFragment(String tableName, Path uri, long start, long length, String[] hosts, Integer[] diskIds) { + super(BuiltinFragmentKinds.FILE, uri.toUri(), tableName, start, start + length, length, hosts); + this.diskIds = diskIds; } + // Non splittable public FileFragment(String tableName, Path uri, long start, long length, String[] hosts) { - this.set(tableName, uri, start, length, hosts, null); + this(tableName, uri, start, length, hosts, null); } public FileFragment(String fragmentId, Path path, long start, long length) { - this.set(fragmentId, path, start, length, null, null); - } - - public FileFragment(FileFragmentProto proto) { - init(proto); - } - - private void init(FileFragmentProto proto) { - int[] diskIds = new int[proto.getDiskIdsList().size()]; - int i = 0; - for(Integer eachValue: proto.getDiskIdsList()) { - diskIds[i++] = eachValue; - } - List var = proto.getHostsList(); - this.set(proto.getId(), new Path(proto.getPath()), - proto.getStartOffset(), proto.getLength(), - var.toArray(new String[var.size()]), - diskIds); - } - - private void set(String tableName, Path path, long start, - long length, String[] hosts, int[] diskIds) { - this.tableName = tableName; - this.uri = path; - this.startOffset = start; - this.length = length; - this.hosts = hosts; - this.diskIds = diskIds; - } - - - /** - * Get the list of hosts (hostname) hosting this block - */ - public String[] getHosts() { - if (hosts == null) { - this.hosts = new String[0]; - } - return hosts; + this(fragmentId, path, start, length, null, null); } /** * Get the list of Disk Ids * Unknown disk is -1. Others 0 ~ N */ - public int[] getDiskIds() { + public Integer[] getDiskIds() { if (diskIds == null) { - this.diskIds = new int[getHosts().length]; - Arrays.fill(this.diskIds, -1); + this.diskIds = new Integer[getHostNames().size()]; + Arrays.fill(this.diskIds, DataLocation.UNKNOWN_VOLUME_ID); } return diskIds; } - public void setDiskIds(int[] diskIds){ + public void setDiskIds(Integer[] diskIds){ this.diskIds = diskIds; } - @Override - public String getTableName() { - return this.tableName; - } - public Path getPath() { - return this.uri; + return new Path(uri); } public void setPath(Path path) { - this.uri = path; - } - - public Long getStartKey() { - return this.startOffset; - } - - @Override - public String getKey() { - return this.uri.toString(); - } - - @Override - public long getLength() { - return this.length; - } - - @Override - public boolean isEmpty() { - return this.length <= 0; - } - /** - * - * The offset range of tablets MUST NOT be overlapped. - * - * @param t - * @return If the table paths are not same, return -1. - */ - @Override - public int compareTo(FileFragment t) { - if (getPath().equals(t.getPath())) { - long diff = this.getStartKey() - t.getStartKey(); - if (diff < 0) { - return -1; - } else if (diff > 0) { - return 1; - } else { - return 0; - } - } else { - return -1; - } + this.uri = path.toUri(); } @Override @@ -191,48 +91,20 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hashCode(tableName, uri, startOffset, length); + return Objects.hashCode(inputSourceId, uri, startKey, endKey, length, diskIds, hostNames); } - + + @Override public Object clone() throws CloneNotSupportedException { FileFragment frag = (FileFragment) super.clone(); - frag.tableName = tableName; - frag.uri = uri; frag.diskIds = diskIds; - frag.hosts = hosts; - return frag; } @Override public String toString() { - return "\"fragment\": {\"id\": \""+ tableName +"\", \"path\": " + return "\"fragment\": {\"id\": \""+ inputSourceId +"\", \"path\": " +getPath() + "\", \"start\": " + this.getStartKey() + ",\"length\": " + getLength() + "}" ; } - - public FragmentProto getProto() { - FileFragmentProto.Builder builder = FileFragmentProto.newBuilder(); - builder.setId(this.tableName); - builder.setStartOffset(this.startOffset); - builder.setLength(this.length); - builder.setPath(this.uri.toString()); - if(diskIds != null) { - List idList = new ArrayList<>(); - for(int eachId: diskIds) { - idList.add(eachId); - } - builder.addAllDiskIds(idList); - } - - if(hosts != null) { - builder.addAllHosts(Arrays.asList(hosts)); - } - - FragmentProto.Builder fragmentBuilder = FragmentProto.newBuilder(); - fragmentBuilder.setId(this.tableName); - fragmentBuilder.setDataFormat(BuiltinStorages.TEXT); - fragmentBuilder.setContents(builder.buildPartial().toByteString()); - return fragmentBuilder.build(); - } } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragmentSerde.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragmentSerde.java new file mode 100644 index 0000000000..926e5febf5 --- /dev/null +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/fragment/FileFragmentSerde.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.fragment; + +import com.google.protobuf.GeneratedMessage.Builder; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.storage.StorageFragmentProtos.FileFragmentProto; + +import java.util.ArrayList; +import java.util.List; + +public class FileFragmentSerde implements FragmentSerde { + + @Override + public Builder newBuilder() { + return FileFragmentProto.newBuilder(); + } + + @Override + public FileFragmentProto serialize(FileFragment fragment) { + FileFragmentProto.Builder builder = FileFragmentProto.newBuilder(); + builder.setId(fragment.inputSourceId); + builder.setStartOffset(fragment.startKey); + builder.setLength(fragment.length); + builder.setPath(fragment.getPath().toString()); + if(fragment.getDiskIds() != null) { + List idList = new ArrayList<>(); + for(int eachId: fragment.getDiskIds()) { + idList.add(eachId); + } + builder.addAllDiskIds(idList); + } + + if(fragment.hostNames != null) { + builder.addAllHosts(fragment.hostNames); + } + return builder.build(); + } + + @Override + public FileFragment deserialize(FileFragmentProto proto) { + return new FileFragment( + proto.getId(), + new Path(proto.getPath()), + proto.getStartOffset(), + proto.getLength(), + proto.getHostsList().toArray(new String[proto.getHostsCount()]), + proto.getDiskIdsList().toArray(new Integer[proto.getDiskIdsCount()])); + } +} diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java index 5e35239a84..caee46cfe5 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineDeserializer.java @@ -194,14 +194,6 @@ private void getValue(JSONObject object, output.put(fieldIndex, DatumFactory.createBlob(Base64.decodeBase64((String) jsonObject))); break; } - case INET4: - String inetStr = object.getAsString(fieldName); - if (inetStr != null) { - output.put(fieldIndex, DatumFactory.createInet4(inetStr)); - } else { - output.put(fieldIndex, NullDatum.get()); - } - break; case RECORD: JSONObject nestedObject = (JSONObject) object.get(fieldName); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java index 9d145737f4..ab5af05f12 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/json/JsonLineSerializer.java @@ -107,7 +107,6 @@ private void putValue(JSONObject json, break; case CHAR: - case INET4: case DATE: case INTERVAL: json.put(fieldName, input.asDatum(fieldIndex).asChars()); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java index 1329955942..0a20d669d0 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java @@ -300,8 +300,8 @@ public void reset() throws IOException { public void close() throws IOException { if (recordReader != null) { recordReader.close(); - tableStats.setNumBytes(recordReader.getNumBytes()); - tableStats.setNumRows(recordCount); + inputStats.setNumBytes(recordReader.getNumBytes()); + inputStats.setNumRows(recordCount); } } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/package-info.java index a987bb9603..9a9f5d609f 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/package-info.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/package-info.java @@ -67,10 +67,6 @@ * BINARY * * - * INET4 - * INTEGER - * - * * TIMESTAMP * TIMESTAMP * diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java index c41f1074b9..7c5b0918c4 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoRecordConverter.java @@ -114,10 +114,6 @@ private Converter newConverter(Column column, Type type, return new FieldFloat4Converter(parent); case FLOAT8: return new FieldFloat8Converter(parent); - case INET4: - return new FieldInet4Converter(parent); - case INET6: - throw new RuntimeException("No converter for INET6"); case DATE: return new FieldDateConverter(parent); case TEXT: @@ -314,19 +310,6 @@ final public void addDouble(double value) { } } - static final class FieldInet4Converter extends PrimitiveConverter { - private final ParentValueContainer parent; - - public FieldInet4Converter(ParentValueContainer parent) { - this.parent = parent; - } - - @Override - final public void addBinary(Binary value) { - parent.add(DatumFactory.createInet4(value.getBytes())); - } - } - static final class FieldDateConverter extends PrimitiveConverter { private final ParentValueContainer parent; diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java index 9fa9428f17..c4d413010c 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoSchemaConverter.java @@ -187,10 +187,6 @@ private Type convertColumn(Column column) { case BLOB: return primitive(column.getSimpleName(), PrimitiveTypeName.BINARY); - case INET4: - case INET6: - return primitive(column.getSimpleName(), - PrimitiveTypeName.BINARY); default: throw new RuntimeException("Cannot convert Tajo type: " + type); } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java index f65b52b483..2009b10e5f 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/TajoWriteSupport.java @@ -147,8 +147,6 @@ private void writeValue(Column column, Tuple tuple, int index) { break; case PROTOBUF: case BLOB: - case INET4: - case INET6: recordConsumer.addBinary(Binary.fromByteArray(tuple.getBytes(index))); break; default: diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java index d7d16b7842..a8eb12d187 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/parquet/package-info.java @@ -75,10 +75,6 @@ * BLOB * BINARY * - * - * INET4 - * BINARY - * * * *

diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rawfile/DirectRawFileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rawfile/DirectRawFileScanner.java index 1e2380e2eb..d876e36c91 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rawfile/DirectRawFileScanner.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rawfile/DirectRawFileScanner.java @@ -171,9 +171,9 @@ public void reset() throws IOException { @Override public void close() throws IOException { - if (tableStats != null) { - tableStats.setReadBytes(filePosition - fragment.getStartKey()); - tableStats.setNumRows(recordCount); + if (inputStats != null) { + inputStats.setReadBytes(filePosition - fragment.getStartKey()); + inputStats.setNumRows(recordCount); } if(tupleBuffer != null) { tupleBuffer.release(); @@ -206,12 +206,12 @@ public boolean isSplittable(){ @Override public TableStats getInputStats() { - if(tableStats != null){ - tableStats.setNumRows(recordCount); - tableStats.setReadBytes(filePosition - fragment.getStartKey()); // actual read bytes (scan + rescan * n) - tableStats.setNumBytes(fragment.getLength()); + if(inputStats != null){ + inputStats.setNumRows(recordCount); + inputStats.setReadBytes(filePosition - fragment.getStartKey()); // actual read bytes (scan + rescan * n) + inputStats.setNumBytes(fragment.getLength()); } - return tableStats; + return inputStats; } @Override diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java index ebdc4722a6..2d21928343 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/rcfile/RCFile.java @@ -1591,9 +1591,9 @@ protected void currentValueBuffer() throws IOException { currentValue.inited = true; readBytes += currentValue.getReadBytes(); - if (tableStats != null) { - tableStats.setReadBytes(readBytes); - tableStats.setNumRows(passedRowsNum); + if (inputStats != null) { + inputStats.setReadBytes(readBytes); + inputStats.setNumRows(passedRowsNum); } } @@ -1776,9 +1776,9 @@ public boolean isSplittable() { @Override public void close() throws IOException { - if (tableStats != null) { - tableStats.setReadBytes(readBytes); //Actual Processed Bytes. (decompressed bytes + header - seek) - tableStats.setNumRows(passedRowsNum); + if (inputStats != null) { + inputStats.setReadBytes(readBytes); //Actual Processed Bytes. (decompressed bytes + header - seek) + inputStats.setNumRows(passedRowsNum); } IOUtils.cleanup(LOG, in, currentValue); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java index 2a3e3028a5..5d8aa83e64 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileAppender.java @@ -83,17 +83,34 @@ public void init() throws IOException { this.fs = path.getFileSystem(conf); - this.delimiter = StringEscapeUtils.unescapeJava(this.meta.getProperty(StorageConstants.SEQUENCEFILE_DELIMITER, + // Set value of non-deprecated key for backward compatibility. + if (!meta.containsProperty(StorageConstants.TEXT_DELIMITER) + && meta.containsProperty(StorageConstants.SEQUENCEFILE_DELIMITER)) { + this.delimiter = StringEscapeUtils.unescapeJava(meta.getProperty(StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0); - this.columnNum = schema.size(); - String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getProperty(StorageConstants.SEQUENCEFILE_NULL, + } else { + this.delimiter = StringEscapeUtils.unescapeJava(meta.getProperty(StorageConstants.TEXT_DELIMITER, + StorageConstants.DEFAULT_FIELD_DELIMITER)).charAt(0); + } + + String nullCharacters; + if (!meta.containsProperty(StorageConstants.TEXT_NULL) + && meta.containsProperty(StorageConstants.SEQUENCEFILE_NULL)) { + nullCharacters = StringEscapeUtils.unescapeJava(meta.getProperty(StorageConstants.SEQUENCEFILE_NULL, NullDatum.DEFAULT_TEXT)); + } else { + nullCharacters = StringEscapeUtils.unescapeJava(meta.getProperty(StorageConstants.TEXT_NULL, + NullDatum.DEFAULT_TEXT)); + } + if (StringUtils.isEmpty(nullCharacters)) { nullChars = NullDatum.get().asTextBytes(); } else { nullChars = nullCharacters.getBytes(); } + this.columnNum = schema.size(); + if(this.meta.containsProperty(StorageConstants.COMPRESSION_CODEC)) { String codecName = this.meta.getProperty(StorageConstants.COMPRESSION_CODEC); codecFactory = new CompressionCodecFactory(conf); @@ -170,7 +187,6 @@ public void addTuple(Tuple tuple) throws IOException { BytesUtils.writeVLong(os, protobufDatum.asByteArray().length); break; case CHAR: - case INET4: case BLOB: BytesUtils.writeVLong(os, tuple.getBytes(j).length); break; diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java index 194d6a91a4..0227ac2d95 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/sequencefile/SequenceFileScanner.java @@ -32,6 +32,7 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.NullDatum; +import org.apache.tajo.exception.TajoInternalError; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.plan.expr.EvalNode; @@ -96,7 +97,28 @@ public void init() throws IOException { reader = new SequenceFile.Reader(fs, fragment.getPath(), conf); - String nullCharacters = StringEscapeUtils.unescapeJava(this.meta.getProperty(StorageConstants.SEQUENCEFILE_NULL, + // Set value of non-deprecated key for backward compatibility. + TableMeta tableMeta; + try { + tableMeta = (TableMeta) meta.clone(); + + if (!tableMeta.containsProperty(StorageConstants.TEXT_DELIMITER)) { + tableMeta.putProperty(StorageConstants.TEXT_DELIMITER, tableMeta.getProperty(StorageConstants + .SEQUENCEFILE_DELIMITER)); + } + + if (!tableMeta.containsProperty(StorageConstants.TEXT_NULL) && tableMeta.containsProperty(StorageConstants + .SEQUENCEFILE_NULL)) { + tableMeta.putProperty(StorageConstants.TEXT_NULL, tableMeta.getProperty(StorageConstants.SEQUENCEFILE_NULL)); + } + } catch (CloneNotSupportedException e) { + throw new TajoInternalError(e); + } + + String delim = tableMeta.getProperty(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); + this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0); + + String nullCharacters = StringEscapeUtils.unescapeJava(tableMeta.getProperty(StorageConstants.TEXT_NULL, NullDatum.DEFAULT_TEXT)); if (StringUtils.isEmpty(nullCharacters)) { nullChars = NullDatum.get().asTextBytes(); @@ -104,9 +126,6 @@ public void init() throws IOException { nullChars = nullCharacters.getBytes(); } - String delim = meta.getProperty(StorageConstants.SEQUENCEFILE_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER); - this.delimiter = StringEscapeUtils.unescapeJava(delim).charAt(0); - this.start = fragment.getStartKey(); this.end = start + fragment.getLength(); @@ -120,9 +139,6 @@ public void init() throws IOException { } outTuple = new VTuple(targets.length); - deserializer = DelimitedTextFile.getLineSerde(meta).createDeserializer(schema, meta, targets); - deserializer.init(); - fieldIsNull = new boolean[schema.getRootColumns().size()]; fieldStart = new int[schema.getRootColumns().size()]; fieldLength = new int[schema.getRootColumns().size()]; @@ -130,12 +146,16 @@ public void init() throws IOException { prepareProjection(targets); try { - String serdeClass = this.meta.getProperty(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName()); + String serdeClass = tableMeta.getProperty(StorageConstants.SEQUENCEFILE_SERDE, + TextSerializerDeserializer.class.getName()); serde = (SerializerDeserializer) Class.forName(serdeClass).newInstance(); serde.init(schema); if (serde instanceof BinarySerializerDeserializer) { hasBinarySerDe = true; + } else { + deserializer = DelimitedTextFile.getLineSerde(tableMeta).createDeserializer(schema, tableMeta, targets); + deserializer.init(); } Class keyClass = (Class)Class.forName(reader.getKeyClassName()); @@ -307,7 +327,6 @@ private void parse(Column col, byte[] bytes, int offset) throws break; case BLOB: case PROTOBUF: - case INET4: case CHAR: case TEXT: elementOffset = 1; @@ -331,9 +350,9 @@ public void close() throws IOException { if (reader != null) reader.close(); - if (tableStats != null) { - tableStats.setReadBytes(totalBytes); - tableStats.setNumRows(currentIdx); + if (inputStats != null) { + inputStats.setReadBytes(totalBytes); + inputStats.setNumRows(currentIdx); } outTuple = null; diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java index 964a58ab53..c383eba99f 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedLineReader.java @@ -36,7 +36,7 @@ import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.storage.*; import org.apache.tajo.storage.compress.CodecPool; -import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.storage.fragment.AbstractFileFragment; import org.apache.tajo.unit.StorageUnit; import java.io.*; @@ -57,15 +57,16 @@ public class DelimitedLineReader implements Closeable { private boolean eof = true; private ByteBufLineReader lineReader; private AtomicInteger lineReadBytes = new AtomicInteger(); - private FileFragment fragment; + private AbstractFileFragment fragment; private Configuration conf; private int bufferSize; - public DelimitedLineReader(Configuration conf, final FileFragment fragment) throws IOException { + public DelimitedLineReader(Configuration conf, final AbstractFileFragment fragment) throws IOException { this(conf, fragment, 128 * StorageUnit.KB); } - public DelimitedLineReader(Configuration conf, final FileFragment fragment, int bufferSize) throws IOException { + public DelimitedLineReader(Configuration conf, final AbstractFileFragment fragment, int bufferSize) + throws IOException { this.fragment = fragment; this.conf = conf; this.factory = new CompressionCodecFactory(conf); @@ -88,7 +89,7 @@ public void init() throws IOException { } pos = startOffset = fragment.getStartKey(); - end = startOffset + fragment.getLength(); + end = fragment.getEndKey(); if (codec != null) { fis = fs.open(fragment.getPath()); @@ -115,13 +116,13 @@ public void init() throws IOException { channel.position(startOffset); is = inputStream; lineReader = new ByteBufLineReader(new LocalFileInputChannel(inputStream), - BufferPool.directBuffer((int) Math.min(bufferSize, end))); + BufferPool.directBuffer((int) Math.min(bufferSize, fragment.getLength()))); } else { fis = fs.open(fragment.getPath()); fis.seek(startOffset); is = fis; lineReader = new ByteBufLineReader(new FSDataInputChannel(fis), - BufferPool.directBuffer((int) Math.min(bufferSize, end))); + BufferPool.directBuffer((int) Math.min(bufferSize, fragment.getLength()))); } } eof = false; @@ -145,7 +146,7 @@ public long getCompressedPosition() throws IOException { return retVal; } - public long getUnCompressedPosition() throws IOException { + public long getUncompressedPosition() throws IOException { return pos; } diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java index e112b0daf5..08ce7bd71e 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/DelimitedTextFile.java @@ -291,7 +291,7 @@ public DelimitedTextFileScanner(Configuration conf, final Schema schema, final T } startOffset = this.fragment.getStartKey(); - endOffset = startOffset + fragment.getLength(); + endOffset = this.fragment.getEndKey(); errorTorrenceMaxNum = Integer.parseInt(meta.getProperty(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM)); @@ -348,59 +348,52 @@ public Tuple next() throws IOException { return null; } - try { - - // this loop will continue until one tuple is build or EOS (end of stream). - do { - long offset = reader.getUnCompressedPosition(); - ByteBuf buf = reader.readLine(); - - // if no more line, then return EOT (end of tuple) - if (buf == null) { - return null; - } + // this loop will continue until one tuple is build or EOS (end of stream). + do { + long offset = reader.getUncompressedPosition(); + ByteBuf buf = reader.readLine(); - // If there is no required column, we just read each line - // and then return an empty tuple without parsing line. - if (targets.length == 0) { - recordCount++; - return EmptyTuple.get(); - } + // if no more line, then return EOT (end of tuple) + if (buf == null) { + return null; + } - outTuple.setOffset(offset); + // If there is no required column, we just read each line + // and then return an empty tuple without parsing line. + if (targets.length == 0) { + recordCount++; + return EmptyTuple.get(); + } - try { - deserializer.deserialize(buf, outTuple); - // if a line is read normally, it exits this loop. - break; + outTuple.setOffset(offset); - } catch (TextLineParsingError tae) { + try { + deserializer.deserialize(buf, outTuple); + // if a line is read normally, it exits this loop. + break; - errorNum++; + } catch (TextLineParsingError tae) { - // suppress too many log prints, which probably cause performance degradation - if (errorNum < errorPrintOutMaxNum) { - LOG.warn("Ignore Text Parse Error (" + errorNum + "): ", tae); - } + errorNum++; - // Only when the maximum error torrence limit is set (i.e., errorTorrenceMaxNum >= 0), - // it checks if the number of parsing error exceeds the max limit. - // Otherwise, it will ignore all parsing errors. - if (errorTorrenceMaxNum >= 0 && errorNum > errorTorrenceMaxNum) { - throw tae; - } + // suppress too many log prints, which probably cause performance degradation + if (errorNum < errorPrintOutMaxNum) { + LOG.warn("Ignore Text Parse Error (" + errorNum + "): ", tae); } - } while (reader.isReadable()); // continue until EOS - // recordCount means the number of actual read records. We increment the count here. - recordCount++; + // Only when the maximum error torrence limit is set (i.e., errorTorrenceMaxNum >= 0), + // it checks if the number of parsing error exceeds the max limit. + // Otherwise, it will ignore all parsing errors. + if (errorTorrenceMaxNum >= 0 && errorNum > errorTorrenceMaxNum) { + throw new IOException(tae); + } + } + } while (reader.isReadable()); // continue until EOS - return outTuple; + // recordCount means the number of actual read records. We increment the count here. + recordCount++; - } catch (Throwable t) { - LOG.error(t); - throw new IOException(t); - } + return outTuple; } @Override @@ -453,8 +446,8 @@ public void close() throws IOException { } if (reader != null) { - tableStats.setReadBytes(reader.getReadBytes()); //Actual Processed Bytes. (decompressed bytes + overhead) - tableStats.setNumRows(recordCount); + inputStats.setReadBytes(reader.getReadBytes()); //Actual Processed Bytes. (decompressed bytes + overhead) + inputStats.setNumRows(recordCount); } if (LOG.isDebugEnabled()) { @@ -488,17 +481,17 @@ public boolean isSplittable() { @Override public TableStats getInputStats() { - if (tableStats != null && reader != null) { - tableStats.setReadBytes(reader.getReadBytes()); //Actual Processed Bytes. (decompressed bytes + overhead) - tableStats.setNumRows(recordCount); - tableStats.setNumBytes(fragment.getLength()); + if (inputStats != null && reader != null) { + inputStats.setReadBytes(reader.getReadBytes()); //Actual Processed Bytes. (decompressed bytes + overhead) + inputStats.setNumRows(recordCount); + inputStats.setNumBytes(fragment.getLength()); } - return tableStats; + return inputStats; } @Override public long getNextOffset() throws IOException { - return reader.getUnCompressedPosition(); + return reader.getUncompressedPosition(); } @Override diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java index f1600c0677..1ca30996d1 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextFieldSerializerDeserializer.java @@ -115,7 +115,6 @@ public int serialize(int columnIndex, Tuple tuple, OutputStream out, byte[] null case INT8: case FLOAT4: case FLOAT8: - case INET4: case DATE: case INTERVAL: bytes = tuple.getTextBytes(columnIndex); @@ -135,7 +134,6 @@ public int serialize(int columnIndex, Tuple tuple, OutputStream out, byte[] null length = bytes.length; out.write(bytes); break; - case INET6: case BLOB: bytes = Base64.encodeBase64(tuple.getBytes(columnIndex), false); length = bytes.length; @@ -237,10 +235,6 @@ public Datum deserialize(int columnIndex, ByteBuf buf, ByteBuf nullChars) throws } break; } - case INET4: - datum = DatumFactory.createInet4( - decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); - break; case BLOB: { byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java index 94a0ba0d09..36183b7ad6 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/text/TextLineSerDe.java @@ -43,13 +43,7 @@ public TextLineSerDe() { public abstract TextLineSerializer createSerializer(Schema schema, TableMeta meta); public static ByteBuf getNullChars(TableMeta meta) { - byte[] nullCharByteArray; - if (meta.getDataFormat().equals(BuiltinStorages.SEQUENCE_FILE)) { - nullCharByteArray = getNullCharsAsBytes(meta, StorageConstants.SEQUENCEFILE_NULL, "\\"); - } else { - nullCharByteArray = getNullCharsAsBytes(meta); - } - + byte[] nullCharByteArray = getNullCharsAsBytes(meta); ByteBuf nullChars = BufferPool.directBuffer(nullCharByteArray.length, nullCharByteArray.length); nullChars.writeBytes(nullCharByteArray); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java index 58fea2bf94..6faeaf6034 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java @@ -32,7 +32,7 @@ import org.apache.tajo.catalog.SchemaBuilder; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; -import org.apache.tajo.storage.fragment.FileFragment; +import org.apache.tajo.storage.fragment.AbstractFileFragment; import org.apache.tajo.storage.thirdparty.orc.TreeReaderFactory.DatumTreeReader; import java.io.Closeable; @@ -71,7 +71,7 @@ public OrcRecordReader(List stripes, FileSystem fileSystem, Schema schema, Column[] targets, - FileFragment fragment, + AbstractFileFragment fragment, List types, CompressionCodec codec, int bufferSize, @@ -99,7 +99,7 @@ public OrcRecordReader(List stripes, long rows = 0; long skippedRows = 0; long offset = fragment.getStartKey(); - long maxOffset = fragment.getStartKey() + fragment.getLength(); + long maxOffset = fragment.getEndKey(); for(StripeInformation stripe: stripes) { long stripeStart = stripe.getOffset(); if (offset > stripeStart) { diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java index b8d3f52c67..e7648b0d43 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java @@ -83,7 +83,6 @@ public static TypeDescription convertTypeInfo(TypeDesc desc) { case INT2: return TypeDescription.createShort(); case INT4: - case INET4: return TypeDescription.createInt(); case INT8: return TypeDescription.createLong(); diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java index b22254dd84..3099779b64 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java @@ -378,60 +378,6 @@ void skipRows(long items) throws IOException { } } - public static class InetTreeReader extends DatumTreeReader { - protected IntegerReader reader = null; - - InetTreeReader(int columnId) throws IOException { - this(columnId, null, null, null); - } - - protected InetTreeReader(int columnId, InStream present, InStream data, - OrcProto.ColumnEncoding encoding) - throws IOException { - super(columnId, present); - if (data != null && encoding != null) { - checkEncoding(encoding); - this.reader = createIntegerReader(encoding.getKind(), data, true, false); - } - } - - @Override - void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException { - if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) && - (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) { - throw new IOException("Unknown encoding " + encoding + " in column " + - columnId); - } - } - - @Override - void startStripe(Map streams, - OrcProto.StripeFooter stripeFooter - ) throws IOException { - super.startStripe(streams, stripeFooter); - org.apache.orc.impl.StreamName name = new org.apache.orc.impl.StreamName(columnId, - OrcProto.Stream.Kind.DATA); - reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(), - streams.get(name), true, false); - } - - @Override - void seek(PositionProvider[] index) throws IOException { - seek(index[columnId]); - } - - @Override - Datum next() throws IOException { - super.next(); - return valuePresent ? DatumFactory.createInet4((int) reader.next()) : NullDatum.get(); - } - - @Override - void skipRows(long items) throws IOException { - reader.skip(countNonNulls(items)); - } - } - public static class IntTreeReader extends DatumTreeReader { protected IntegerReader reader = null; @@ -1541,8 +1487,6 @@ public static DatumTreeReader createTreeReader(TimeZone timeZone, return new TimestampTreeReader(timeZone, orcColumnId, skipCorrupt); case DATE: return new DateTreeReader(orcColumnId); - case INET4: - return new InetTreeReader(orcColumnId); // case STRUCT: // return new StructTreeReader(columnId, treeReaderSchema, included, skipCorrupt); default: diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java index 1ff32b88d6..5dedc2eae8 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java @@ -37,7 +37,6 @@ import org.apache.orc.OrcUtils; import org.apache.orc.impl.*; import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.Inet4Datum; import org.apache.tajo.datum.Int4Datum; import org.apache.tajo.datum.Int8Datum; import org.apache.tajo.storage.Tuple; @@ -964,7 +963,7 @@ void write(Datum datum) throws IOException { super.write(datum); if (datum != null && datum.isNotNull()) { long val; - if (datum instanceof Int4Datum || datum instanceof Inet4Datum) { + if (datum instanceof Int4Datum) { val = datum.asInt4(); } else if (datum instanceof Int8Datum) { val = datum.asInt8(); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java index 0c8695c749..ad7df24984 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java @@ -59,7 +59,6 @@ public class TestDelimitedTextFile { .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) .add("col9", Type.BLOB) - .add("col10", Type.INET4) .build(); baseTuple = new VTuple(new Datum[] { @@ -72,7 +71,6 @@ public class TestDelimitedTextFile { DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 DatumFactory.createBlob("hyunsik".getBytes()),// 8 - DatumFactory.createInet4("192.168.0.1"), // 9 }); } @@ -120,7 +118,7 @@ public void testIncompleteQuote() throws IOException, CloneNotSupportedException Tuple tuple; int i = 0; while ((tuple = scanner.next()) != null) { - assertEquals("(f,hyunsik\",NULL,NULL,NULL,NULL,0.0,\"hyunsik,hyunsik,NULL)", tuple.toString()); + assertEquals("(f,hyunsik\",NULL,NULL,NULL,NULL,0.0,\"hyunsik,hyunsik)", tuple.toString()); i++; } assertEquals(1, i); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java index c62a01cf26..18183300fa 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileSystems.java @@ -127,7 +127,7 @@ public void testBlockSplit() throws IOException { appender.close(); FileStatus fileStatus = fs.getFileStatus(path); - List splits = sm.getSplits("table", meta, schema, path); + List splits = sm.getSplits("table", meta, schema, false, path); int splitSize = (int) Math.ceil(fileStatus.getLen() / (double) fileStatus.getBlockSize()); assertEquals(splitSize, splits.size()); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java index dc8781ef38..42b0b2321b 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestFileTablespace.java @@ -156,17 +156,17 @@ public void testGetSplit() throws Exception { List splits = Lists.newArrayList(); // Get FileFragments in partition batch - splits.addAll(space.getSplits("data", meta, schema, partitions.toArray(new Path[partitions.size()]))); + splits.addAll(space.getSplits("data", meta, schema, false, partitions.toArray(new Path[partitions.size()]))); assertEquals(testCount, splits.size()); // -1 is unknown volumeId - assertEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]); + assertEquals(DataLocation.UNKNOWN_VOLUME_ID, ((FileFragment)splits.get(0)).getDiskIds()[0].intValue()); splits.clear(); - splits.addAll(space.getSplits("data", meta, schema, + splits.addAll(space.getSplits("data", meta, schema, false, partitions.subList(0, partitions.size() / 2).toArray(new Path[partitions.size() / 2]))); assertEquals(testCount / 2, splits.size()); - assertEquals(1, splits.get(0).getHosts().length); - assertEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]); + assertEquals(1, splits.get(0).getHostNames().size()); + assertEquals(DataLocation.UNKNOWN_VOLUME_ID, ((FileFragment)splits.get(0)).getDiskIds()[0].intValue()); fs.close(); } finally { @@ -212,7 +212,7 @@ public void testZeroLengthSplit() throws Exception { List splits = Lists.newArrayList(); // Get FileFragments in partition batch - splits.addAll(space.getSplits("data", meta, schema, partitions.toArray(new Path[partitions.size()]))); + splits.addAll(space.getSplits("data", meta, schema, false, partitions.toArray(new Path[partitions.size()]))); assertEquals(0, splits.size()); fs.close(); @@ -256,12 +256,12 @@ public void testGetSplitWithBlockStorageLocationsBatching() throws Exception { TableMeta meta = CatalogUtil.newTableMeta(BuiltinStorages.TEXT, conf); List splits = Lists.newArrayList(); - splits.addAll(sm.getSplits("data", meta, schema, tablePath)); + splits.addAll(sm.getSplits("data", meta, schema, false, tablePath)); assertEquals(testCount, splits.size()); - assertEquals(2, splits.get(0).getHosts().length); + assertEquals(2, splits.get(0).getHostNames().size()); assertEquals(2, ((FileFragment)splits.get(0)).getDiskIds().length); - assertNotEquals(-1, ((FileFragment)splits.get(0)).getDiskIds()[0]); + assertNotEquals(DataLocation.UNKNOWN_VOLUME_ID, ((FileFragment)splits.get(0)).getDiskIds()[0].intValue()); fs.close(); } finally { diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java index d8d77754f3..deb758d94d 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java @@ -269,7 +269,7 @@ public void testZeroRows() throws IOException { assertEquals(0, fileStatus.getLen()); } - List splits = sm.getSplits("testZeroRows", meta, schema, testDir); + List splits = sm.getSplits("testZeroRows", meta, schema, false, testDir); int tupleCnt = 0; for (Fragment fragment : splits) { Scanner scanner = sm.getScanner(meta, schema, fragment, schema); @@ -414,13 +414,12 @@ public void testVariousTypes() throws IOException { .add("col6", Type.FLOAT4) .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) - .add("col9", Type.BLOB) - .add("col10", Type.INET4); + .add("col9", Type.BLOB); if (dateTypeSupport()) { - schemaBld.add("col11", Type.DATE); + schemaBld.add("col10", Type.DATE); } if (protoTypeSupport()) { - schemaBld.add("col12", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); + schemaBld.add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); } Schema schema = schemaBld.build(); @@ -438,7 +437,7 @@ public void testVariousTypes() throws IOException { QueryId queryid = new QueryId("12345", 5); - VTuple tuple = new VTuple(10 + (dateTypeSupport() ? 1 : 0) + (protoTypeSupport() ? 1 : 0)); + VTuple tuple = new VTuple(9 + (dateTypeSupport() ? 1 : 0) + (protoTypeSupport() ? 1 : 0)); tuple.put(new Datum[] { DatumFactory.createBool(true), DatumFactory.createChar("hyunsik"), @@ -449,10 +448,9 @@ public void testVariousTypes() throws IOException { DatumFactory.createFloat8(271.9f), DatumFactory.createText("hyunsik"), DatumFactory.createBlob("hyunsik".getBytes()), - DatumFactory.createInet4("192.168.0.1"), }); - short currentIdx = 10; + short currentIdx = 9; if (dateTypeSupport()) { tuple.put(currentIdx, DatumFactory.createDate(2016, 6, 28)); @@ -492,11 +490,10 @@ public void testNullHandlingTypes() throws IOException { .add("col6", Type.FLOAT4) .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) - .add("col9", Type.BLOB) - .add("col10", Type.INET4); + .add("col9", Type.BLOB); if (protoTypeSupport()) { - schemaBld.add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); + schemaBld.add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); } Schema schema = schemaBld.build(); @@ -517,7 +514,7 @@ public void testNullHandlingTypes() throws IOException { QueryId queryid = new QueryId("12345", 5); ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName()); - int columnNum = 10 + (protoTypeSupport() ? 1 : 0); + int columnNum = 9 + (protoTypeSupport() ? 1 : 0); VTuple seedTuple = new VTuple(columnNum); seedTuple.put(new Datum[]{ DatumFactory.createBool(true), // 0 @@ -529,11 +526,10 @@ public void testNullHandlingTypes() throws IOException { DatumFactory.createFloat8(271.9f), // 7 DatumFactory.createText("hyunsik"), // 8 DatumFactory.createBlob("hyunsik".getBytes()),// 9 - DatumFactory.createInet4("192.168.0.1") // 10 }); if (protoTypeSupport()) { - seedTuple.put(10, factory.createDatum(queryid.getProto())); // 11 + seedTuple.put(9, factory.createDatum(queryid.getProto())); // 10 } // Making tuples with different null column positions @@ -587,11 +583,10 @@ public void testNullHandlingTypesWithProjection() throws IOException { .add("col6", Type.FLOAT4) .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) - .add("col9", Type.BLOB) - .add("col10", Type.INET4); + .add("col9", Type.BLOB); if (protoTypeSupport()) { - schemaBld.add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); + schemaBld.add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())); } Schema schema = schemaBld.build(); @@ -612,7 +607,7 @@ public void testNullHandlingTypesWithProjection() throws IOException { QueryId queryid = new QueryId("12345", 5); ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName()); - int columnNum = 10 + (protoTypeSupport() ? 1 : 0); + int columnNum = 9 + (protoTypeSupport() ? 1 : 0); VTuple seedTuple = new VTuple(columnNum); seedTuple.put(new Datum[]{ DatumFactory.createBool(true), // 0 @@ -624,11 +619,10 @@ public void testNullHandlingTypesWithProjection() throws IOException { DatumFactory.createFloat8(271.9f), // 7 DatumFactory.createText("hyunsik"), // 8 DatumFactory.createBlob("hyunsik".getBytes()),// 9 - DatumFactory.createInet4("192.168.0.1") // 10 }); if (protoTypeSupport()) { - seedTuple.put(10, factory.createDatum(queryid.getProto())); // 11 + seedTuple.put(9, factory.createDatum(queryid.getProto())); // 10 } // Making tuples with different null column positions @@ -695,8 +689,7 @@ public void testRCFileTextSerializeDeserialize() throws IOException { .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) .add("col9", Type.BLOB) - .add("col10", Type.INET4) - .add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) .build(); TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); @@ -721,7 +714,6 @@ public void testRCFileTextSerializeDeserialize() throws IOException { DatumFactory.createFloat8(271.9f), DatumFactory.createText("jinho"), DatumFactory.createBlob("hyunsik babo".getBytes()), - DatumFactory.createInet4("192.168.0.1"), factory.createDatum(queryid.getProto()) }); appender.addTuple(tuple); @@ -760,8 +752,7 @@ public void testRCFileBinarySerializeDeserialize() throws IOException { .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) .add("col9", Type.BLOB) - .add("col10", Type.INET4) - .add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) .build(); TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); @@ -786,7 +777,6 @@ public void testRCFileBinarySerializeDeserialize() throws IOException { DatumFactory.createFloat8(271.9f), DatumFactory.createText("jinho"), DatumFactory.createBlob("hyunsik babo".getBytes()), - DatumFactory.createInet4("192.168.0.1"), factory.createDatum(queryid.getProto()) }); appender.addTuple(tuple); @@ -825,8 +815,7 @@ public void testSequenceFileTextSerializeDeserialize() throws IOException { .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) .add("col9", Type.BLOB) - .add("col10", Type.INET4) - .add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())).build(); + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())).build(); TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); meta.putProperty(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName()); @@ -849,7 +838,6 @@ public void testSequenceFileTextSerializeDeserialize() throws IOException { DatumFactory.createFloat8(271.9f), DatumFactory.createText("jinho"), DatumFactory.createBlob("hyunsik babo".getBytes()), - DatumFactory.createInet4("192.168.0.1"), ProtobufDatumFactory.createDatum(queryid.getProto()) }); appender.addTuple(tuple); @@ -878,6 +866,140 @@ public void testSequenceFileTextSerializeDeserialize() throws IOException { assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue()); } + @Test + public void testSequenceFileTextSerializeDeserializeWithDeprecatedProperties() throws IOException { + if(!dataFormat.equalsIgnoreCase(BuiltinStorages.SEQUENCE_FILE)) return; + + Schema schema = SchemaBuilder.builder() + .add("col1", Type.BOOLEAN) + .add("col2", CatalogUtil.newDataTypeWithLen(Type.CHAR, 7)) + .add("col3", Type.INT2) + .add("col4", Type.INT4) + .add("col5", Type.INT8) + .add("col6", Type.FLOAT4) + .add("col7", Type.FLOAT8) + .add("col8", Type.TEXT) + .add("col9", Type.BLOB) + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())).build(); + + TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); + meta.putProperty(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName()); + meta.putProperty(StorageConstants.SEQUENCEFILE_DELIMITER, "\u0001"); + meta.putProperty(StorageConstants.SEQUENCEFILE_NULL, "\\"); + + Path tablePath = new Path(testDir, "testSequenceFileTextSerializeDeserializeWithDeprecatedProperties.data"); + FileTablespace sm = TablespaceManager.getLocalFs(); + Appender appender = sm.getAppender(meta, schema, tablePath); + appender.enableStats(); + appender.init(); + + QueryId queryid = new QueryId("12345", 5); + + VTuple tuple = new VTuple(new Datum[] { + DatumFactory.createBool(true), + DatumFactory.createChar("jinho"), + DatumFactory.createInt2((short) 17), + DatumFactory.createInt4(59), + DatumFactory.createInt8(23l), + DatumFactory.createFloat4(77.9f), + DatumFactory.createFloat8(271.9f), + DatumFactory.createText("jinho"), + DatumFactory.createBlob("hyunsik babo".getBytes()), + ProtobufDatumFactory.createDatum(queryid.getProto()) + }); + appender.addTuple(tuple); + appender.flush(); + appender.close(); + + FileStatus status = fs.getFileStatus(tablePath); + assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen()); + + FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen()); + Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment, null); + scanner.init(); + + assertTrue(scanner instanceof SequenceFileScanner); + Writable key = ((SequenceFileScanner) scanner).getKey(); + assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName()); + + Tuple retrieved; + while ((retrieved=scanner.next()) != null) { + for (int i = 0; i < tuple.size(); i++) { + assertEquals(tuple.get(i), retrieved.asDatum(i)); + } + } + scanner.close(); + assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue()); + assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue()); + } + + @Test + public void testSequenceFileTextSerializeDeserializeWithNonDeprecatedProperties() throws IOException { + if(!dataFormat.equalsIgnoreCase(BuiltinStorages.SEQUENCE_FILE)) return; + + Schema schema = SchemaBuilder.builder() + .add("col1", Type.BOOLEAN) + .add("col2", CatalogUtil.newDataTypeWithLen(Type.CHAR, 7)) + .add("col3", Type.INT2) + .add("col4", Type.INT4) + .add("col5", Type.INT8) + .add("col6", Type.FLOAT4) + .add("col7", Type.FLOAT8) + .add("col8", Type.TEXT) + .add("col9", Type.BLOB) + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())).build(); + + TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); + meta.putProperty(StorageConstants.SEQUENCEFILE_SERDE, TextSerializerDeserializer.class.getName()); + meta.putProperty(StorageConstants.TEXT_DELIMITER, "\u0001"); + meta.putProperty(StorageConstants.TEXT_NULL, "\\"); + + Path tablePath = new Path(testDir, "testSequenceFileTextSerializeDeserializeWithNonDeprecatedProperties.data"); + FileTablespace sm = TablespaceManager.getLocalFs(); + Appender appender = sm.getAppender(meta, schema, tablePath); + appender.enableStats(); + appender.init(); + + QueryId queryid = new QueryId("12345", 5); + + VTuple tuple = new VTuple(new Datum[] { + DatumFactory.createBool(true), + DatumFactory.createChar("jinho"), + DatumFactory.createInt2((short) 17), + DatumFactory.createInt4(59), + DatumFactory.createInt8(23l), + DatumFactory.createFloat4(77.9f), + DatumFactory.createFloat8(271.9f), + DatumFactory.createText("jinho"), + DatumFactory.createBlob("hyunsik babo".getBytes()), + ProtobufDatumFactory.createDatum(queryid.getProto()) + }); + appender.addTuple(tuple); + appender.flush(); + appender.close(); + + FileStatus status = fs.getFileStatus(tablePath); + assertEquals(appender.getStats().getNumBytes().longValue(), status.getLen()); + + FileFragment fragment = new FileFragment("table", tablePath, 0, status.getLen()); + Scanner scanner = TablespaceManager.getLocalFs().getScanner(meta, schema, fragment, null); + scanner.init(); + + assertTrue(scanner instanceof SequenceFileScanner); + Writable key = ((SequenceFileScanner) scanner).getKey(); + assertEquals(key.getClass().getCanonicalName(), LongWritable.class.getCanonicalName()); + + Tuple retrieved; + while ((retrieved=scanner.next()) != null) { + for (int i = 0; i < tuple.size(); i++) { + assertEquals(tuple.get(i), retrieved.asDatum(i)); + } + } + scanner.close(); + assertEquals(appender.getStats().getNumBytes().longValue(), scanner.getInputStats().getNumBytes().longValue()); + assertEquals(appender.getStats().getNumRows().longValue(), scanner.getInputStats().getNumRows().longValue()); + } + @Test public void testSequenceFileBinarySerializeDeserialize() throws IOException { if(!dataFormat.equalsIgnoreCase(BuiltinStorages.SEQUENCE_FILE)) return; @@ -892,8 +1014,7 @@ public void testSequenceFileBinarySerializeDeserialize() throws IOException { .add("col7", Type.FLOAT8) .add("col8", Type.TEXT) .add("col9", Type.BLOB) - .add("col10", Type.INET4) - .add("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) + .add("col10", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName())) .build(); TableMeta meta = CatalogUtil.newTableMeta(dataFormat, conf); @@ -917,7 +1038,6 @@ public void testSequenceFileBinarySerializeDeserialize() throws IOException { DatumFactory.createFloat8(271.9f), DatumFactory.createText("jinho"), DatumFactory.createBlob("hyunsik babo".getBytes()), - DatumFactory.createInet4("192.168.0.1"), ProtobufDatumFactory.createDatum(queryid.getProto()) }); appender.addTuple(tuple); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java index 0e6fde5bdd..7a9b074176 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.tajo.BuiltinStorages; import org.apache.tajo.catalog.*; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.conf.TajoConf; @@ -74,8 +75,8 @@ public TestBSTIndex(String type) { @Parameters(name = "{index}: {0}") public static Collection generateParameters() { return Arrays.asList(new Object[][]{ - {"RAW"}, - {"TEXT"} + {BuiltinStorages.RAW}, + {BuiltinStorages.TEXT} }); } @@ -128,7 +129,7 @@ public void testFindValue() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -152,7 +153,7 @@ public void testFindValue() throws IOException { BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + dataFormat + ".idx"), keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); for (int i = 0; i < TUPLE_NUM - 1; i++) { @@ -233,7 +234,7 @@ public void testBuildIndexWithAppender() throws IOException { keySchema, comp); reader.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); for (int i = 0; i < TUPLE_NUM - 1; i++) { @@ -298,7 +299,7 @@ public void testFindOmittedValue() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -372,7 +373,7 @@ public void testFindNextKeyValue() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -396,7 +397,7 @@ public void testFindNextKeyValue() throws IOException { keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple result; @@ -466,7 +467,7 @@ public void testFindNextKeyOmittedValue() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -490,7 +491,7 @@ public void testFindNextKeyOmittedValue() throws IOException { keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple result; @@ -549,7 +550,7 @@ public void testFindMinValue() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -575,7 +576,7 @@ public void testFindMinValue() throws IOException { keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); tuple.put(0, DatumFactory.createInt8(0)); @@ -636,7 +637,7 @@ public void testMinMax() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -744,7 +745,7 @@ public void testConcurrentAccess() throws IOException, InterruptedException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -828,7 +829,7 @@ public void testFindValueDescOrder() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -854,7 +855,7 @@ public void testFindValueDescOrder() throws IOException { keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); for (int i = (TUPLE_NUM - 1); i > 0; i--) { @@ -921,7 +922,7 @@ public void testFindNextKeyValueDescOrder() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -950,7 +951,7 @@ public void testFindNextKeyValueDescOrder() throws IOException { assertEquals(comp, reader.getComparator()); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple result; @@ -1023,7 +1024,7 @@ public void testFindValueASCOrder() throws IOException { creater.init(); SeekableScanner scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); Tuple keyTuple; @@ -1047,7 +1048,7 @@ public void testFindValueASCOrder() throws IOException { BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "testFindValue_" + dataFormat + ".idx"), keySchema, comp); reader.init(); scanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()). - getSeekableScanner(meta, schema, tablet.getProto(), schema); + getSeekableScanner(meta, schema, tablet, schema); scanner.init(); for (int i = 0; i < TUPLE_NUM - 1; i++) { diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java index 4f6b566e86..cca7920a67 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java @@ -114,7 +114,7 @@ public void testFindValueInSingleCSV() throws IOException { creater.init(); SeekableScanner fileScanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()) - .getSeekableScanner(meta, schema, tablet.getProto(), schema); + .getSeekableScanner(meta, schema, tablet, schema); fileScanner.init(); Tuple keyTuple; long offset; @@ -139,7 +139,7 @@ public void testFindValueInSingleCSV() throws IOException { "FindValueInCSV.idx"), keySchema, comp); reader.init(); fileScanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()) - .getSeekableScanner(meta, schema, tablet.getProto(), schema); + .getSeekableScanner(meta, schema, tablet, schema); fileScanner.init(); for (int i = 0; i < TUPLE_NUM - 1; i++) { tuple.put(0, DatumFactory.createInt8(i)); @@ -206,7 +206,7 @@ public void testFindNextKeyValueInSingleCSV() throws IOException { creater.init(); SeekableScanner fileScanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()) - .getSeekableScanner(meta, schema, tablet.getProto(), schema); + .getSeekableScanner(meta, schema, tablet, schema); fileScanner.init(); Tuple keyTuple; long offset; @@ -228,7 +228,7 @@ public void testFindNextKeyValueInSingleCSV() throws IOException { BSTIndexReader reader = bst.getIndexReader(new Path(testDir, "FindNextKeyValueInCSV.idx"), keySchema, comp); reader.init(); fileScanner = OldStorageManager.getStorageManager(conf, meta.getDataFormat()) - .getSeekableScanner(meta, schema, tablet.getProto(), schema); + .getSeekableScanner(meta, schema, tablet, schema); fileScanner.init(); Tuple result; for(int i = 0 ; i < TUPLE_NUM -1 ; i ++) { diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java index 8cf45861be..a4d424b554 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java @@ -57,8 +57,7 @@ public class TestJsonSerDe { .add("col7", TajoDataTypes.Type.FLOAT8) .add("col8", TajoDataTypes.Type.TEXT) .add("col9", TajoDataTypes.Type.BLOB) - .add("col10", TajoDataTypes.Type.INET4) - .add("col11", TajoDataTypes.Type.NULL_TYPE) + .add("col10", TajoDataTypes.Type.NULL_TYPE) .build(); } @@ -89,13 +88,12 @@ public void testVarioutType() throws IOException { DatumFactory.createChar("hyunsik"), // 1 DatumFactory.createInt2((short) 17), // 2 DatumFactory.createInt4(59), // 3 - DatumFactory.createInt8(23l), // 4 + DatumFactory.createInt8(23L), // 4 DatumFactory.createFloat4(77.9f), // 5 DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 DatumFactory.createBlob("hyunsik".getBytes()), // 8 - DatumFactory.createInet4("192.168.0.1"), // 9 - NullDatum.get(), // 10 + NullDatum.get(), // 9 }); assertEquals(baseTuple, tuple); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java index 77b58942c7..488f3f12ff 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/parquet/TestSchemaConverter.java @@ -56,7 +56,6 @@ public class TestSchemaConverter { " optional binary mytext (UTF8);\n" + " optional binary myblob;\n" + // NULL_TYPE fields are not encoded. - " optional binary myinet4;\n" + " optional binary myprotobuf;\n" + "}\n"; @@ -72,7 +71,6 @@ private Schema createAllTypesSchema() { .add(new Column("mytext", Type.TEXT)) .add(new Column("myblob", Type.BLOB)) .add(new Column("mynull", Type.NULL_TYPE)) - .add(new Column("myinet4", Type.INET4)) .add(new Column("myprotobuf", Type.PROTOBUF)) .build(); } diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/raw/TestDirectRawFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/raw/TestDirectRawFile.java index 569438e1f5..395b159db1 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/raw/TestDirectRawFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/raw/TestDirectRawFile.java @@ -140,8 +140,7 @@ public Path getTestDir(FileSystem fs, String dir) throws IOException { .add("col8", TajoDataTypes.Type.DATE) .add("col9", TajoDataTypes.Type.TIME) .add("col10", TajoDataTypes.Type.INTERVAL) - .add("col11", TajoDataTypes.Type.INET4) - .add("col12", + .add("col11", CatalogUtil.newDataType(TajoDataTypes.Type.PROTOBUF, PrimitiveProtos.StringProto.class.getName())) .build(); } @@ -293,8 +292,7 @@ public static void fillRow(int i, RowWriter builder) { builder.putDate(DatumFactory.createDate("2014-04-16").asInt4() + i); // 8 builder.putTime(DatumFactory.createTime("08:48:00").asInt8() + i); // 9 builder.putInterval(DatumFactory.createInterval((i + 1) + " hours")); // 10 - builder.putInet4(DatumFactory.createInet4("192.168.0.1").asInt4() + i); // 11 - builder.putProtoDatum(new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 12 + builder.putProtoDatum(new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 11 builder.endRow(); } @@ -310,8 +308,7 @@ public static void validateTupleResult(int j, Tuple t) { assertEquals(DatumFactory.createDate("2014-04-16").asInt4() + j, t.getInt4(8)); assertEquals(DatumFactory.createTime("08:48:00").asInt8() + j, t.getInt8(9)); assertEquals(DatumFactory.createInterval((j + 1) + " hours"), t.getInterval(10)); - assertEquals(DatumFactory.createInet4("192.168.0.1").asInt4() + j, t.getInt4(11)); - assertEquals(new ProtobufDatum(ProtoUtil.convertString(j + "")), t.getProtobufDatum(12)); + assertEquals(new ProtobufDatum(ProtoUtil.convertString(j + "")), t.getProtobufDatum(11)); } public static void fillRowBlockWithNull(int i, RowWriter writer) { @@ -384,12 +381,6 @@ public static void fillRowBlockWithNull(int i, RowWriter writer) { if (i % 11 == 0) { writer.skipField(); - } else { - writer.putInet4(DatumFactory.createInet4("192.168.0.1").asInt4() + i); // 11 - } - - if (i % 12 == 0) { - writer.skipField(); } else { writer.putProtoDatum(new ProtobufDatum(ProtoUtil.convertString(i + ""))); // 12 } @@ -465,12 +456,6 @@ public static void validateNullity(int j, Tuple tuple) { } if (j % 11 == 0) { - tuple.isBlankOrNull(11); - } else { - assertEquals(DatumFactory.createInet4("192.168.0.1").asInt4() + j, tuple.getInt4(11)); - } - - if (j % 12 == 0) { tuple.isBlankOrNull(12); } else { assertEquals(new ProtobufDatum(ProtoUtil.convertString(j + "")), tuple.getProtobufDatum(12)); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc index f1d1368447..0255e00f55 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc +++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc @@ -11,8 +11,7 @@ { "name": "col6", "type": "float" }, { "name": "col7", "type": "double" }, { "name": "col8", "type": "string" }, - { "name": "col9", "type": "bytes" }, - { "name": "col10", "type": "bytes" } + { "name": "col9", "type": "bytes" } ] } diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml index 3283f9faab..7ae58aaf84 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml +++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml @@ -39,44 +39,28 @@ - tajo.storage.fragment.text.class + tajo.storage.fragment.kind.file org.apache.tajo.storage.fragment.FileFragment - tajo.storage.fragment.json.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.hbase + org.apache.tajo.storage.hbase.HBaseFragment - tajo.storage.fragment.raw.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.kind.jdbc + org.apache.tajo.storage.jdbc.JdbcFragment - tajo.storage.fragment.draw.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.file + org.apache.tajo.storage.fragment.FileFragmentSerde - tajo.storage.fragment.rcfile.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.hbase + org.apache.tajo.storage.hbase.HBaseFragmentSerde - tajo.storage.fragment.row.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.parquet.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.orc.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.sequencefile.class - org.apache.tajo.storage.fragment.FileFragment - - - tajo.storage.fragment.avro.class - org.apache.tajo.storage.fragment.FileFragment + tajo.storage.fragment.serde.jdbc + org.apache.tajo.storage.jdbc.JdbcFragmentSerde diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragment.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragment.java index 0088504c5a..20c1acad5c 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragment.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragment.java @@ -18,90 +18,34 @@ package org.apache.tajo.storage.jdbc; -import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; -import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.TajoConstants; +import org.apache.tajo.storage.fragment.BuiltinFragmentKinds; import org.apache.tajo.storage.fragment.Fragment; -import org.apache.tajo.storage.jdbc.JdbcFragmentProtos.JdbcFragmentProto; -import java.util.Arrays; +import java.net.URI; +import java.util.List; -public class JdbcFragment implements Fragment, Comparable, Cloneable { - String uri; - String inputSourceId; - String [] hostNames; - - - public JdbcFragment(ByteString raw) throws InvalidProtocolBufferException { - JdbcFragmentProto.Builder builder = JdbcFragmentProto.newBuilder(); - builder.mergeFrom(raw); - builder.build(); - init(builder.build()); - } +/** + * Fragment for the systems which connects to Tajo via the JDBC interface. + */ +public class JdbcFragment extends Fragment { - public JdbcFragment(String inputSourceId, String uri) { - this.inputSourceId = inputSourceId; - this.uri = uri; - this.hostNames = extractHosts(uri); + // TODO: set start and end keys properly + public JdbcFragment(String inputSourceId, URI uri) { + super(BuiltinFragmentKinds.JDBC, uri, inputSourceId, null, null, TajoConstants.UNKNOWN_LENGTH, extractHosts(uri)); } - private void init(JdbcFragmentProto proto) { - this.uri = proto.getUri(); - this.inputSourceId = proto.getInputSourceId(); - this.hostNames = proto.getHostsList().toArray(new String [proto.getHostsCount()]); + public JdbcFragment(String inputSourceId, URI uri, List hostNames) { + super(BuiltinFragmentKinds.JDBC, uri, inputSourceId, null, null, TajoConstants.UNKNOWN_LENGTH, + hostNames.toArray(new String[hostNames.size()])); } - private String [] extractHosts(String uri) { + private static String[] extractHosts(URI uri) { return new String[] {ConnectionInfo.fromURI(uri).host}; } - @Override - public String getTableName() { - return inputSourceId; - } - - public String getUri() { - return uri; - } - - @Override - public long getLength() { - return 0; - } - - @Override - public String getKey() { - return null; - } - - @Override - public String[] getHosts() { - return hostNames; - } - @Override public boolean isEmpty() { return false; } - - @Override - public CatalogProtos.FragmentProto getProto() { - JdbcFragmentProto.Builder builder = JdbcFragmentProto.newBuilder(); - builder.setInputSourceId(this.inputSourceId); - builder.setUri(this.uri); - if(hostNames != null) { - builder.addAllHosts(Arrays.asList(hostNames)); - } - - CatalogProtos.FragmentProto.Builder fragmentBuilder = CatalogProtos.FragmentProto.newBuilder(); - fragmentBuilder.setId(this.inputSourceId); - fragmentBuilder.setDataFormat("JDBC"); - fragmentBuilder.setContents(builder.buildPartial().toByteString()); - return fragmentBuilder.build(); - } - - @Override - public int compareTo(JdbcFragment o) { - return this.uri.compareTo(o.uri); - } } diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragmentSerde.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragmentSerde.java new file mode 100644 index 0000000000..9e26d80eba --- /dev/null +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcFragmentSerde.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.jdbc; + +import com.google.protobuf.GeneratedMessage.Builder; +import org.apache.tajo.storage.fragment.FragmentSerde; +import org.apache.tajo.storage.jdbc.JdbcFragmentProtos.JdbcFragmentProto; + +import java.net.URI; + +public class JdbcFragmentSerde implements FragmentSerde { + + @Override + public Builder newBuilder() { + return JdbcFragmentProto.newBuilder(); + } + + @Override + public JdbcFragmentProto serialize(JdbcFragment fragment) { + return JdbcFragmentProto.newBuilder() + .setInputSourceId(fragment.getInputSourceId()) + .setUri(fragment.getUri().toASCIIString()) + .addAllHosts(fragment.getHostNames()) + .build(); + } + + @Override + public JdbcFragment deserialize(JdbcFragmentProto proto) { + return new JdbcFragment(proto.getInputSourceId(), URI.create(proto.getUri()), proto.getHostsList()); + } +} diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java index 8f5e7a8665..9b712a818e 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcMetadataProviderBase.java @@ -30,6 +30,7 @@ import org.apache.tajo.exception.TajoInternalError; import org.apache.tajo.exception.UndefinedTablespaceException; import org.apache.tajo.exception.UnsupportedDataTypeException; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.Pair; @@ -197,7 +198,7 @@ public TableDesc getTableDesc(String schemaName, String tableName) throws Undefi final String qualifier = resultForColumns.getString("TABLE_NAME"); final String columnName = resultForColumns.getString("COLUMN_NAME"); final TypeDesc type = convertDataType(resultForColumns); - final Column c = new Column(CatalogUtil.buildFQName(databaseName, qualifier, columnName), type); + final Column c = new Column(IdentifierUtil.buildFQName(databaseName, qualifier, columnName), type); columns.add(new Pair<>(ordinalPos, c)); } @@ -223,11 +224,12 @@ public Column apply(@Nullable Pair columnPair) { final TableStats stats = new TableStats(); stats.setNumRows(-1); // unknown + TableMeta meta = new TableMeta("rowstore", new KeyValueSet()); final TableDesc table = new TableDesc( - CatalogUtil.buildFQName(databaseName, name), + IdentifierUtil.buildFQName(databaseName, name), schema, - new TableMeta("rowstore", new KeyValueSet()), - space.getTableUri(databaseName, name) + meta, + space.getTableUri(meta, databaseName, name) ); table.setStats(stats); diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java index d7e1c3b926..76ed3ab3dd 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcScanner.java @@ -36,6 +36,7 @@ import org.apache.tajo.storage.Scanner; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; +import org.apache.tajo.util.datetime.DateTimeUtil; import java.io.Closeable; import java.io.IOException; @@ -231,7 +232,9 @@ protected void convertTuple(ResultSet resultSet, VTuple tuple) { tuple.put(column_idx, DatumFactory.createDate(1900 + date.getYear(), 1 + date.getMonth(), date.getDate())); break; case TIME: - tuple.put(column_idx, new TimeDatum(resultSet.getTime(resultIdx).getTime() * 1000)); + final Time time = resultSet.getTime(resultIdx); + tuple.put(column_idx, new TimeDatum( + DateTimeUtil.toTime(time.getHours(), time.getMinutes(), time.getSeconds(), 0))); break; case TIMESTAMP: tuple.put(column_idx, @@ -255,7 +258,7 @@ protected void convertTuple(ResultSet resultSet, VTuple tuple) { private ResultSetIterator executeQueryAndGetIter() { try { LOG.info("Generated SQL: " + generatedSql); - Connection conn = DriverManager.getConnection(fragment.uri, connProperties); + Connection conn = DriverManager.getConnection(fragment.getUri().toASCIIString(), connProperties); Statement statement = conn.createStatement(); ResultSet resultset = statement.executeQuery(generatedSql); return new ResultSetIterator((resultset)); diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcTablespace.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcTablespace.java index fa6cf486e2..21bce9c416 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcTablespace.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/JdbcTablespace.java @@ -110,20 +110,21 @@ protected void storageInit() throws IOException { } @Override - public long getTableVolume(TableDesc table, Optional filter) throws UnsupportedException { - throw new UnsupportedException(); + public long getTableVolume(TableDesc table, Optional filter) { + throw new TajoRuntimeException(new UnsupportedException()); } @Override - public URI getTableUri(String databaseName, String tableName) { + public URI getTableUri(TableMeta meta, String databaseName, String tableName) { return URI.create(UriUtil.addParam(getUri().toASCIIString(), URI_PARAM_KEY_TABLE, tableName)); } @Override public List getSplits(String inputSourceId, TableDesc tableDesc, + boolean requireSorted, @Nullable EvalNode filterCondition) throws IOException { - return Lists.newArrayList((Fragment)new JdbcFragment(inputSourceId, tableDesc.getUri().toASCIIString())); + return Lists.newArrayList((Fragment)new JdbcFragment(inputSourceId, tableDesc.getUri())); } @Override diff --git a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java index 0b1df31054..064d6cb61a 100644 --- a/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java +++ b/tajo-storage/tajo-storage-jdbc/src/main/java/org/apache/tajo/storage/jdbc/SQLExpressionGenerator.java @@ -19,13 +19,12 @@ package org.apache.tajo.storage.jdbc; import com.google.common.base.Function; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.NotImplementedException; import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UnsupportedDataTypeException; import org.apache.tajo.plan.expr.*; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.type.Type; import org.apache.tajo.util.StringUtils; @@ -145,13 +144,13 @@ public String apply(Datum v) { protected EvalNode visitField(Context context, FieldEval field, Stack stack) { // strip the database name String tableName; - if (CatalogUtil.isSimpleIdentifier(field.getQualifier())) { + if (IdentifierUtil.isSimpleIdentifier(field.getQualifier())) { tableName = field.getQualifier(); } else { - tableName = CatalogUtil.extractSimpleName(field.getQualifier()); + tableName = IdentifierUtil.extractSimpleName(field.getQualifier()); } - context.append(CatalogUtil.buildFQName(tableName, field.getColumnName())); + context.append(IdentifierUtil.buildFQName(tableName, field.getColumnName())); return field; } @@ -230,7 +229,7 @@ protected EvalNode visitSubquery(Context context, SubqueryEval evalNode, Stack + + + + + tajo-project + org.apache.tajo + 0.12.0-SNAPSHOT + ../../tajo-project + + 4.0.0 + + tajo-storage-mysql + jar + Tajo MySQL JDBC storage + + UTF-8 + UTF-8 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + ${project.build.sourceEncoding} + + + + org.apache.rat + apache-rat-plugin + + + derby.log + src/test/resources/dataset/** + src/test/resources/queries/** + src/test/resources/results/** + src/test/resources/mysql/** + + + + + verify + + check + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + + + + + + org.apache.tajo + tajo-common + provided + + + org.apache.tajo + tajo-catalog-common + provided + + + org.apache.tajo + tajo-plan + provided + + + org.apache.tajo + tajo-storage-common + provided + + + org.apache.tajo + tajo-storage-hdfs + provided + + + org.apache.tajo + tajo-storage-jdbc + provided + + + org.apache.tajo + tajo-cluster-tests + test-jar + test + + + + org.apache.hadoop + hadoop-common + provided + + + org.apache.hadoop + hadoop-minicluster + test + + + commons-el + commons-el + + + tomcat + jasper-runtime + + + tomcat + jasper-compiler + + + org.mortbay.jetty + jsp-2.1-jetty + + + com.sun.jersey.jersey-test-framework + jersey-test-framework-grizzly2 + + + netty-all + io.netty + + + + + org.apache.hadoop + hadoop-hdfs + test + + + commons-el + commons-el + + + tomcat + jasper-runtime + + + tomcat + jasper-compiler + + + org.mortbay.jetty + jsp-2.1-jetty + + + com.sun.jersey.jersey-test-framework + jersey-test-framework-grizzly2 + + + netty-all + io.netty + + + + + + com.google.protobuf + protobuf-java + + + junit + junit + test + + + io.airlift + testing-postgresql-server + 0.3 + test + + + com.google.guava + guava + 15.0 + test + + + + io.airlift + testing-mysql-server + test + 0.1 + + + + + + + + test-storage-mysql + + + + org.apache.maven.plugins + maven-surefire-plugin + + + TRUE + + -Xms128m -Xmx1024m -Dfile.encoding=UTF-8 + + + + + + + docs + + false + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + module-javadocs + package + + jar + + + ${project.build.directory} + + + + + + + + + + + + + org.apache.maven.plugins + maven-surefire-report-plugin + + + + diff --git a/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLJdbcScanner.java b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLJdbcScanner.java new file mode 100644 index 0000000000..5c9d41f1c4 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLJdbcScanner.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.storage.jdbc.JdbcFragment; +import org.apache.tajo.storage.jdbc.JdbcScanner; + +import java.sql.DatabaseMetaData; +import java.util.Properties; + +public class MySQLJdbcScanner extends JdbcScanner { + + public MySQLJdbcScanner(DatabaseMetaData dbMetaData, + Properties connProperties, + Schema tableSchema, + TableMeta tableMeta, + JdbcFragment fragment) { + super(dbMetaData, connProperties, tableSchema, tableMeta, fragment); + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLMetadataProvider.java b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLMetadataProvider.java new file mode 100644 index 0000000000..48aed38266 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLMetadataProvider.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import org.apache.tajo.storage.jdbc.JdbcMetadataProviderBase; + +import javax.annotation.Nullable; +import java.util.Collection; + +public class MySQLMetadataProvider extends JdbcMetadataProviderBase { + + public MySQLMetadataProvider(MySQLTablespace space, String dbName) { + super(space, dbName); + } + + @Override + protected String getJdbcDriverName() { + return "com.mysql.jdbc.Driver"; + } + + @Override + public Collection getTables(@Nullable String schemaPattern, @Nullable String tablePattern) { + return super.getTables("public", tablePattern); + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLTablespace.java b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLTablespace.java new file mode 100644 index 0000000000..ba37d27c15 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/main/java/org/apache/tajo/storage/mysql/MySQLTablespace.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + + +import net.minidev.json.JSONObject; +import org.apache.tajo.catalog.*; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UndefinedTableException; +import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.schema.IdentifierUtil; +import org.apache.tajo.storage.NullScanner; +import org.apache.tajo.storage.Scanner; +import org.apache.tajo.storage.fragment.Fragment; +import org.apache.tajo.storage.jdbc.JdbcFragment; +import org.apache.tajo.storage.jdbc.JdbcTablespace; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.net.URI; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Optional; + +/** + * MySQL Database Tablespace + */ +public class MySQLTablespace extends JdbcTablespace { + + public MySQLTablespace(String name, URI uri, JSONObject config) { + super(name, uri, config); + } + + public MetadataProvider getMetadataProvider() { + return new MySQLMetadataProvider(this, database); + } + + @Override + public Scanner getScanner(TableMeta meta, + Schema schema, + Fragment fragment, + @Nullable Schema target) throws IOException { + if (!(fragment instanceof JdbcFragment)) { + throw new TajoInternalError("fragment must be JdbcFragment"); + } + + if (target == null) { + target = schema; + } + + Scanner scanner; + if (fragment.isEmpty()) { + scanner = new NullScanner(conf, schema, meta, fragment); + } else { + scanner = new MySQLJdbcScanner(getDatabaseMetaData(), connProperties, schema, meta, (JdbcFragment) fragment); + } + scanner.setTarget(target.toArray()); + return scanner; + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException(); + } + + @Override + public long getTableVolume(TableDesc table, Optional filter) { + String databaseName = IdentifierUtil.extractQualifier(table.getName()); + String tableName = IdentifierUtil.extractSimpleName(table.getName()); + + String sql = "SELECT sum(data_length + index_length) as volume FROM information_schema.tables " + + " WHERE table_schema = '" + databaseName + "' AND table_name = '" + tableName + "'"; + + try (Statement stmt = conn.createStatement(); + ResultSet rs = stmt.executeQuery(sql)) { + if (rs.next()) { + return rs.getLong(1); + } else { + throw new TajoRuntimeException(new UndefinedTableException(table.getName())); + } + } catch (SQLException e) { + throw new TajoInternalError(e); + } + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/MySQLTestServer.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/MySQLTestServer.java new file mode 100644 index 0000000000..9157a72e57 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/MySQLTestServer.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import io.airlift.testing.mysql.TestingMySqlServer; +import net.minidev.json.JSONObject; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.jdbc.JdbcTablespace; +import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.FileUtil; +import org.apache.tajo.util.JavaResourceUtil; + +import java.io.IOException; +import java.net.URI; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Map; +import java.util.Optional; + +import static java.lang.String.format; + +public class MySQLTestServer { + private static final Log LOG = LogFactory.getLog(MySQLTestServer.class); + + private static final MySQLTestServer instance; + + public static final String [] TPCH_TABLES = { + "customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier" + }; + + public static final String SPACENAME = "mysql_cluster"; + public static final String DATABASE_NAME = "tpch"; + public static final String USERNAME = "testuser"; + public static final String PASSWORD = "testpass"; + + private final TestingMySqlServer server; + + static { + try { + instance = new MySQLTestServer(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static MySQLTestServer getInstance() { + return instance; + } + + private MySQLTestServer() throws Exception { + server = new TestingMySqlServer(USERNAME, PASSWORD, DATABASE_NAME); + + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + server.close(); + } + })); + + loadTPCHTables(); + registerTablespace(); + } + + Path testPath = CommonTestingUtil.getTestDir(); + + private void loadTPCHTables() throws SQLException, IOException { + + try (Connection connection = DriverManager.getConnection(getJdbcUrl())) { + try (Statement statement = connection.createStatement()) { + statement.execute(format("GRANT ALL ON %s.* TO '%s'@'%%' IDENTIFIED BY '%s'", DATABASE_NAME, "mysql", "mysql")); + + for (String tableName : TPCH_TABLES) { + String sql = JavaResourceUtil.readTextFromResource("mysql/" + tableName + ".sql"); + statement.execute(sql); + + // restore the table contents into a file stored in a local file system for MySQL LOAD DATA command + String path = restoreTableContents(tableName); + String copyCommand = genLoadStatement(tableName, path); + statement.execute(copyCommand); + } + + // load DATETIME_TYPES table + String sql = JavaResourceUtil.readTextFromResource("mysql/datetime_types.sql"); + statement.executeUpdate(sql); + Path filePath = new Path(testPath, "datetime_types.txt"); + storeTableContents("mysql/datetime_types.txt", filePath); + String copyCommand = genLoadStatement("DATETIME_TYPES", filePath.toUri().getPath()); + LOG.info(copyCommand); + statement.execute(copyCommand); + + } catch (Throwable t) { + t.printStackTrace(); + throw t; + } + } + } + + private String genLoadStatement(String tableName, String path) { + return "LOAD DATA INFILE '" + path + "' INTO TABLE " + tableName + " FIELDS TERMINATED BY '|'"; + } + + private void storeTableContents(String resource, Path path) throws IOException { + String csvTable = JavaResourceUtil.readTextFromResource(resource); + String fixedCsvTable = fixExtraColumn(csvTable); + FileUtil.writeTextToFile(fixedCsvTable, path); + } + + private String restoreTableContents(String tableName) throws IOException { + Path filePath = new Path(testPath, tableName + ".tbl"); + storeTableContents("dataset/" + tableName + ".tbl", filePath); + return filePath.toUri().getPath(); + } + + private String fixExtraColumn(String csvTable) { + final String [] lines = csvTable.split("\n"); + final StringBuilder rewritten = new StringBuilder(); + + for (String l : lines) { + if (l.charAt(l.length() - 1) == '|') { + rewritten.append(l.substring(0, l.length() - 1)); + } else { + rewritten.append(l.substring(0, l.length())); + } + rewritten.append("\n"); + } + + return rewritten.toString(); + } + + private void registerTablespace() throws IOException { + JSONObject configElements = new JSONObject(); + configElements.put(JdbcTablespace.CONFIG_KEY_MAPPED_DATABASE, DATABASE_NAME); + + MySQLTablespace tablespace = new MySQLTablespace(SPACENAME, URI.create(getJdbcUrl()), configElements); + tablespace.init(new TajoConf()); + + TablespaceManager.addTableSpaceForTest(tablespace); + } + + /** + * get JDBC URL for test server + * + * @return JDBC URL + */ + public String getJdbcUrl() { + String[] url = server.getJdbcUrl().split("\\?"); + return url[0] + "/" + DATABASE_NAME + "?" + url[1]; + } + + public TestingMySqlServer getServer() { + return server; + } + + public static Optional resetAllParamsAndSetConnProperties(Map connProperties) + throws IOException { + String uri = MySQLTestServer.getInstance().getJdbcUrl().split("\\?")[0]; + + JSONObject configElements = new JSONObject(); + configElements.put(JdbcTablespace.CONFIG_KEY_MAPPED_DATABASE, MySQLTestServer.DATABASE_NAME); + + JSONObject connPropertiesJson = new JSONObject(); + for (Map.Entry entry : connProperties.entrySet()) { + connPropertiesJson.put(entry.getKey(), entry.getValue()); + } + configElements.put(JdbcTablespace.CONFIG_KEY_CONN_PROPERTIES, connPropertiesJson); + + MySQLTablespace tablespace = new MySQLTablespace(MySQLTestServer.SPACENAME, URI.create(uri), configElements); + tablespace.init(new TajoConf()); + return TablespaceManager.addTableSpaceForTest(tablespace); + } + + public static void main(String[] args) throws Exception { + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLEndPointTests.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLEndPointTests.java new file mode 100644 index 0000000000..b62a07952a --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLEndPointTests.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import com.google.common.collect.Sets; +import org.apache.tajo.QueryTestCaseBase; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.client.TajoClient; +import org.apache.tajo.exception.UndefinedTableException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestMySQLEndPointTests extends QueryTestCaseBase { + private static final String jdbcUrl = MySQLTestServer.getInstance().getJdbcUrl(); + private static TajoClient client; + + + @BeforeClass + public static void setUp() throws Exception { + QueryTestCaseBase.testingCluster.getMaster().refresh(); + client = QueryTestCaseBase.testingCluster.newTajoClient(); + } + + @AfterClass + public static void tearDown() { + client.close(); + } + + @Test(timeout = 1000) + public void testGetAllDatabaseNames() { + Set retrieved = Sets.newHashSet(client.getAllDatabaseNames()); + assertTrue(retrieved.contains(MySQLTestServer.DATABASE_NAME)); + } + + @Test(timeout = 1000) + public void testGetTableList() { + final Set expected = Sets.newHashSet(MySQLTestServer.TPCH_TABLES); + expected.add("DATETIME_TYPES"); + final Set retrieved = Sets.newHashSet(client.getTableList("tpch")); + + assertEquals(expected, retrieved); + } + + @Test(timeout = 1000) + public void testGetTable() throws UndefinedTableException { + for (String tableName: MySQLTestServer.TPCH_TABLES) { + TableDesc retrieved = client.getTableDesc(MySQLTestServer.DATABASE_NAME + "." + tableName); + assertEquals(MySQLTestServer.DATABASE_NAME + "." + tableName, retrieved.getName()); + assertEquals(jdbcUrl + "&table=" + tableName, retrieved.getUri().toASCIIString()); + } + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLMetadataProvider.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLMetadataProvider.java new file mode 100644 index 0000000000..f34f685e2a --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLMetadataProvider.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import com.google.common.collect.Sets; +import org.apache.tajo.catalog.MetadataProvider; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.junit.Test; + +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestMySQLMetadataProvider { + private static final String jdbcUrl = MySQLTestServer.getInstance().getJdbcUrl(); + + @Test + public void testGetTablespaceName() throws Exception { + Tablespace tablespace = TablespaceManager.get(jdbcUrl); + MetadataProvider provider = tablespace.getMetadataProvider(); + assertEquals("mysql_cluster", provider.getTablespaceName()); + } + + @Test + public void testGetDatabaseName() throws Exception { + Tablespace tablespace = TablespaceManager.get(jdbcUrl); + MetadataProvider provider = tablespace.getMetadataProvider(); + assertEquals("tpch", provider.getDatabaseName()); + } + + @Test + public void testGetSchemas() throws Exception { + Tablespace tablespace = TablespaceManager.get(jdbcUrl); + MetadataProvider provider = tablespace.getMetadataProvider(); + assertTrue(provider.getSchemas().isEmpty()); + } + + @Test + public void testGetTables() throws Exception { + Tablespace tablespace = TablespaceManager.get(jdbcUrl); + MetadataProvider provider = tablespace.getMetadataProvider(); + + final Set expected = Sets.newHashSet(MySQLTestServer.TPCH_TABLES); + expected.add("DATETIME_TYPES"); + final Set found = Sets.newHashSet(provider.getTables(null, null)); + + assertEquals(expected, found); + } + + @Test + public void testGetTableDescriptor() throws Exception { + Tablespace tablespace = TablespaceManager.get(jdbcUrl); + MetadataProvider provider = tablespace.getMetadataProvider(); + + for (String tableName : MySQLTestServer.TPCH_TABLES) { + TableDesc table = provider.getTableDesc(null, tableName); + assertEquals("tpch." + tableName, table.getName()); + assertEquals(jdbcUrl + "&table=" + tableName, table.getUri().toASCIIString()); + } + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLQueryTests.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLQueryTests.java new file mode 100644 index 0000000000..e9bbbaef2b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLQueryTests.java @@ -0,0 +1,220 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import org.apache.tajo.QueryTestCaseBase; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class TestMySQLQueryTests extends QueryTestCaseBase { + @SuppressWarnings("unused") + // This should be invoked for initializing MySQLTestServer + private static final String jdbcUrl = MySQLTestServer.getInstance().getJdbcUrl(); + + public TestMySQLQueryTests() { + super(MySQLTestServer.DATABASE_NAME); + } + + @BeforeClass + public static void setUp() { + QueryTestCaseBase.testingCluster.getMaster().refresh(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testProjectedColumns() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testFixedLengthFields() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testVariableLengthFields() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testDateTimeTypes() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testTPCH_Q1() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testTPCH_Q2_Part() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testTPCH_Q2_Part_MixedStorage() throws Exception { + // Manually enable broadcast feature + + try { + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "true"); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_NON_CROSS_JOIN_THRESHOLD.varname, + "" + (5 * 1024)); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_CROSS_JOIN_THRESHOLD.varname, + 1024 * 1024 + ""); + + runSimpleTests(); + + } finally { + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, + TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.defaultVal); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_NON_CROSS_JOIN_THRESHOLD.varname, + TajoConf.ConfVars.$DIST_QUERY_BROADCAST_NON_CROSS_JOIN_THRESHOLD.defaultVal); + testingCluster.setAllTajoDaemonConfValue(TajoConf.ConfVars.$DIST_QUERY_BROADCAST_CROSS_JOIN_THRESHOLD.varname, + 1024 * 1024 + ""); + } + } + + @SimpleTest + @Test + @Option(sort = true) + public void testTPCH_Q3() throws Exception { + runSimpleTests(); + } + +// @SimpleTest +// @Test + // TODO: enable this test after allowing consecutive cross joins (TAJO-2075) + @Option(sort = true) + public void testTPCH_Q5() throws Exception { + runSimpleTests(); + } + + // Predicates -------------------------------------------------------------- + + @SimpleTest + @Test + @Option(sort = true) + public void testSimpleFilter() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testInPredicateWithNumbers() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testInPredicateWithLiterals() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testBetweenNumbers() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testBetweenDates() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testCaseWhenFilter() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testFunctionWithinFilter() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testCountAsterisk() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + @Option(sort = true) + public void testCtasToHdfs() throws Exception { + try { + executeString("CREATE DATABASE mytmp").close(); + executeString("CREATE TABLE mytmp.offload AS SELECT * FROM LINEITEM").close(); + + runSimpleTests(); + + } finally { + executeString("DROP TABLE IF EXISTS mytmp.offload").close(); + executeString("DROP DATABASE IF EXISTS mytmp").close(); + } + } + + @SimpleTest + @Test + @Option(sort = true) + public void testQueryWithConnProperties() throws Exception { + Map connProperties = new HashMap<>(); + connProperties.put("user", "mysql"); + connProperties.put("password", "mysql"); + + Optional old = Optional.empty(); + try { + old = MySQLTestServer.resetAllParamsAndSetConnProperties(connProperties); + runSimpleTests(); + } finally { + if (old.isPresent()) { + TablespaceManager.addTableSpaceForTest(old.get()); + } + } + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSQLJdbcTableSpace.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSQLJdbcTableSpace.java new file mode 100644 index 0000000000..730f867e89 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSQLJdbcTableSpace.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import net.minidev.json.JSONObject; +import org.apache.tajo.catalog.MetadataProvider; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.exception.TajoException; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UndefinedTablespaceException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TablespaceManager; +import org.apache.tajo.storage.fragment.Fragment; +import org.apache.tajo.storage.jdbc.JdbcTablespace; +import org.junit.Test; + +import java.io.IOException; +import java.net.URI; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.Assert.*; + +public class TestMySQLSQLJdbcTableSpace { + private static String jdbcUrl = MySQLTestServer.getInstance().getJdbcUrl(); + + @Test(timeout = 1000) + public void testTablespaceHandler() throws Exception { + assertTrue((TablespaceManager.getByName("mysql_cluster")) instanceof MySQLTablespace); + assertEquals("mysql_cluster", (TablespaceManager.getByName("mysql_cluster").getName())); + + assertTrue((TablespaceManager.get(jdbcUrl)) instanceof MySQLTablespace); + assertTrue((TablespaceManager.get(jdbcUrl + "&table=tb1")) instanceof MySQLTablespace); + + assertEquals(jdbcUrl, TablespaceManager.get(jdbcUrl).getUri().toASCIIString()); + assertTrue(TablespaceManager.get(jdbcUrl).getMetadataProvider() instanceof MySQLMetadataProvider); + } + + @Test(timeout = 1000, expected = TajoRuntimeException.class) + public void testCreateTable() throws IOException, TajoException { + Tablespace space = TablespaceManager.getByName("mysql_cluster"); + space.createTable(null, false); + } + + @Test(timeout = 1000, expected = TajoRuntimeException.class) + public void testDropTable() throws IOException, TajoException { + Tablespace space = TablespaceManager.getByName("mysql_cluster"); + space.purgeTable(null); + } + + @Test(timeout = 1000) + public void testGetSplits() throws IOException, TajoException { + Tablespace space = TablespaceManager.getByName("mysql_cluster"); + MetadataProvider provider = space.getMetadataProvider(); + TableDesc table = provider.getTableDesc(null, "lineitem"); + List fragments = space.getSplits("lineitem", table, false, null); + assertNotNull(fragments); + assertEquals(1, fragments.size()); + } + + @Test + public void testConnProperties() throws Exception { + Map connProperties = new HashMap<>(); + connProperties.put("user", "mysql"); + connProperties.put("password", "mysql"); + + String uri = MySQLTestServer.getInstance().getJdbcUrl().split("\\?")[0]; + Tablespace space = new MySQLTablespace("t1", URI.create(uri), getJsonTablespace(connProperties)); + try { + space.init(new TajoConf()); + } finally { + space.close(); + } + } + + @Test + public void testConnPropertiesNegative() throws Exception { + Map connProperties = new HashMap<>(); + connProperties.put("user", "mysqlX"); + connProperties.put("password", ""); + + String uri = MySQLTestServer.getInstance().getJdbcUrl().split("\\?")[0]; + Tablespace space = new MySQLTablespace("t1", URI.create(uri), getJsonTablespace(connProperties)); + try { + space.init(new TajoConf()); + fail("Access denied for user 'mysqlX'@'localhost' (using password: NO)"); + } catch (IOException ioe) { + assertTrue(ioe.getCause() instanceof SQLException); + } finally { + space.close(); + } + } + + public static JSONObject getJsonTablespace(Map connProperties) + throws IOException { + JSONObject configElements = new JSONObject(); + configElements.put(JdbcTablespace.CONFIG_KEY_MAPPED_DATABASE, MySQLTestServer.DATABASE_NAME); + + JSONObject connPropertiesJson = new JSONObject(); + for (Map.Entry entry : connProperties.entrySet()) { + connPropertiesJson.put(entry.getKey(), entry.getValue()); + } + configElements.put(JdbcTablespace.CONFIG_KEY_CONN_PROPERTIES, connPropertiesJson); + + return configElements; + } + + @Test + public void testTemporaryTablespace() { + Optional ts = TablespaceManager.removeTablespaceForTest("mysql_cluster"); + assertTrue(ts.isPresent()); + + Tablespace tempTs = TablespaceManager.get(jdbcUrl); + assertNotNull(tempTs); + + TablespaceManager.addTableSpaceForTest(ts.get()); + } + + @Test + public void testGetTableVolume() throws UndefinedTablespaceException, UnsupportedException { + Tablespace space = TablespaceManager.getByName("mysql_cluster"); + MetadataProvider provider = space.getMetadataProvider(); + TableDesc table = provider.getTableDesc(null, "lineitem"); + long volume = space.getTableVolume(table, Optional.empty()); + assertEquals(16384L, volume); + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSimpleQueryTests.java b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSimpleQueryTests.java new file mode 100644 index 0000000000..c7082d255b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/java/org/apache/tajo/storage/mysql/TestMySQLSimpleQueryTests.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.mysql; + +import org.apache.tajo.QueryTestCaseBase; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestMySQLSimpleQueryTests extends QueryTestCaseBase { + @SuppressWarnings("unused") + // This should be invoked for initializing MySQLTestServer + private static final String jdbcUrl = MySQLTestServer.getInstance().getJdbcUrl(); + + public TestMySQLSimpleQueryTests() { + super(MySQLTestServer.DATABASE_NAME); + } + + @BeforeClass + public static void setUp() { + QueryTestCaseBase.testingCluster.getMaster().refresh(); + } + + @SimpleTest + @Test + public void testSelectAll() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + public void testSelectLimit() throws Exception { + runSimpleTests(); + } +} diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/.marker b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/.marker new file mode 100644 index 0000000000..158780d799 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/.marker @@ -0,0 +1 @@ +// for keeping dataset directory \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/customer.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/customer.tbl new file mode 100644 index 0000000000..a291700613 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/customer.tbl @@ -0,0 +1,5 @@ +1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e| +2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref| +3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov| +4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou| +5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/lineitem.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/lineitem.tbl new file mode 100644 index 0000000000..b2bb7a7a19 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/lineitem.tbl @@ -0,0 +1,5 @@ +1|1|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the| +1|1|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold | +2|2|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a| +3|2|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco| +3|3|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/nation.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/nation.tbl new file mode 100644 index 0000000000..4f73be3278 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/nation.tbl @@ -0,0 +1,25 @@ +0|ALGERIA|0| haggle. carefully final deposits detect slyly agai| +1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon| +2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special | +3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold| +4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d| +5|ETHIOPIA|0|ven packages wake quickly. regu| +6|FRANCE|3|refully final requests. regular, ironi| +7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco| +8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun| +9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull| +10|IRAN|4|efully alongside of the slyly final dependencies. | +11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula| +12|JAPAN|2|ously. final, express gifts cajole a| +13|JORDAN|4|ic deposits are blithely about the carefully regular pa| +14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t| +15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?| +16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r| +17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun| +18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos| +19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account| +20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely| +21|VIETNAM|2|hely enticingly express accounts. even, final | +22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint| +23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull| +24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/orders.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/orders.tbl new file mode 100644 index 0000000000..e692cfb94e --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/orders.tbl @@ -0,0 +1,3 @@ +1|3|O|173665.47|1996-01-02|5-LOW|Clerk#000000951|0|nstructions sleep furiously among | +2|4|O|46929.18|1996-12-01|1-URGENT|Clerk#000000880|0| foxes. pending accounts at the pending, silent asymptot| +3|2|F|193846.25|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular ideas cajole carefully. depos| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/part.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/part.tbl new file mode 100644 index 0000000000..6e6fa721c9 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/part.tbl @@ -0,0 +1,4 @@ +1|goldenrod lavender spring chocolate lace|Manufacturer#1|Brand#13|PROMO BURNISHED COPPER|7|JUMBO PKG|901.00|ly. slyly ironi +2|blush thistle blue yellow saddle|Manufacturer#1|Brand#13|LARGE BRUSHED BRASS|15|LG CASE|902.00|lar accounts amo +3|spring green yellow purple cornsilk|Manufacturer#4|Brand#42|STANDARD POLISHED BRASS|21|WRAP CASE|903.00|egular deposits hag +4|cornflower chocolate smoke green pink|Manufacturer#3|Brand#34|SMALL PLATED BRASS|14|MED DRUM|904.00|p furiously r \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/partsupp.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/partsupp.tbl new file mode 100644 index 0000000000..a183389b6c --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/partsupp.tbl @@ -0,0 +1,3 @@ +1|2|3325|771.64|, even theodolites. regular, final theodolites eat after the carefully pending foxes. furiously regular deposits sleep slyly. carefully bold realms above the ironic dependencies haggle careful| +2|3|8895|1.01|nic accounts. final accounts sleep furiously about the ironic, bold packages. regular, regular accounts| +3|4|4651|920.92|ilent foxes affix furiously quickly unusual requests. even packages across the carefully even theodolites nag above the sp| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/region.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/region.tbl new file mode 100644 index 0000000000..ef50e31999 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/region.tbl @@ -0,0 +1,5 @@ +0|AFRICA|lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to | +1|AMERICA|hs use ironic, even requests. s| +2|ASIA|ges. thinly even pinto beans ca| +3|EUROPE|ly final courts cajole furiously final excuse| +4|MIDDLE EAST|uickly special accounts cajole carefully blithely close requests. carefully final asymptotes haggle furiousl| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/supplier.tbl b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/supplier.tbl new file mode 100644 index 0000000000..c51ce4eb99 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/dataset/supplier.tbl @@ -0,0 +1,3 @@ +2|Supplier#000000002|89eJ5ksX3ImxJQBvxObC,|5|15-679-861-2259|4032.68| slyly bold instructions. idle dependen| +3|Supplier#000000003|q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3|1|11-383-516-1199|4192.40|blithely silent requests after the express dependencies are sl| +4|Supplier#000000004|Bk7ah4CK8SYQTepEmvMkkgMwg|15|25-843-787-7479|4641.08|riously even requests above the exp| \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/customer.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/customer.sql new file mode 100644 index 0000000000..3ff60ad123 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/customer.sql @@ -0,0 +1,10 @@ +create table customer ( + c_custkey integer not null, + c_name varchar(25) not null, + c_address varchar(40) not null, + c_nationkey integer not null, + c_phone char(15) not null, + c_acctbal decimal(15,2) not null, + c_mktsegment char(10) not null, + c_comment varchar(117) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.sql new file mode 100644 index 0000000000..9cbbc2f889 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.sql @@ -0,0 +1,5 @@ +CREATE TABLE DATETIME_TYPES ( + date_field DATE, + time_field TIME, + ts_field TIMESTAMP +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.txt b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.txt new file mode 100644 index 0000000000..6c7d8ff277 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/datetime_types.txt @@ -0,0 +1,4 @@ +1980-04-01|01:50:27|1980-04-01 01:50:27 +1984-01-05|02:45:31|1984-01-05 02:45:31 +1984-01-15|14:12:58|1984-01-15 14:12:58 +2015-03-21|08:11:01|2015-03-21 08:11:01 \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/lineitem.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/lineitem.sql new file mode 100644 index 0000000000..c7d69cda17 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/lineitem.sql @@ -0,0 +1,18 @@ +create table lineitem ( + l_orderkey integer not null, + l_partkey integer not null, + l_suppkey integer not null, + l_linenumber integer not null, + l_quantity decimal(15,2) not null, + l_extendedprice decimal(15,2) not null, + l_discount decimal(15,2) not null, + l_tax decimal(15,2) not null, + l_returnflag char(1) not null, + l_linestatus char(1) not null, + l_shipdate date not null, + l_commitdate date not null, + l_receiptdate date not null, + l_shipinstruct char(25) not null, + l_shipmode char(10) not null, + l_comment varchar(44) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/nation.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/nation.sql new file mode 100644 index 0000000000..092db0a1c7 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/nation.sql @@ -0,0 +1,6 @@ +CREATE TABLE nation ( + n_nationkey INTEGER NOT NULL, + n_name CHAR(25) NOT NULL, + n_regionkey INTEGER NOT NULL, + n_comment VARCHAR(152) +); diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/orders.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/orders.sql new file mode 100644 index 0000000000..c0ec039e9a --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/orders.sql @@ -0,0 +1,11 @@ +create table orders ( + o_orderkey integer not null, + o_custkey integer not null, + o_orderstatus char(1) not null, + o_totalprice decimal(15,2) not null, + o_orderdate date not null, + o_orderpriority char(15) not null, + o_clerk char(15) not null, + o_shippriority integer not null, + o_comment varchar(79) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/part.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/part.sql new file mode 100644 index 0000000000..3e9fda8ec1 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/part.sql @@ -0,0 +1,11 @@ +create table part ( + p_partkey integer not null, + p_name varchar(55) not null, + p_mfgr char(25) not null, + p_brand char(10) not null, + p_type varchar(25) not null, + p_size integer not null, + p_container char(10) not null, + p_retailprice decimal(15,2) not null, + p_comment varchar(23) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/partsupp.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/partsupp.sql new file mode 100644 index 0000000000..ffb9e9779f --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/partsupp.sql @@ -0,0 +1,7 @@ +create table partsupp ( + ps_partkey integer not null, + ps_suppkey integer not null, + ps_availqty integer not null, + ps_supplycost decimal(15,2) not null, + ps_comment varchar(199) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/region.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/region.sql new file mode 100644 index 0000000000..c9b7f4c038 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/region.sql @@ -0,0 +1,5 @@ +create table region ( + r_regionkey integer not null, + r_name char(25) not null, + r_comment varchar(152) +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/supplier.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/supplier.sql new file mode 100644 index 0000000000..360f8d52ba --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/mysql/supplier.sql @@ -0,0 +1,9 @@ +create table supplier ( + s_suppkey integer not null, + s_name char(25) not null, + s_address varchar(40) not null, + s_nationkey integer not null, + s_phone char(15) not null, + s_acctbal decimal(15,2) not null, + s_comment varchar(101) not null +); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenDates.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenDates.sql new file mode 100644 index 0000000000..119c5ec1e0 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenDates.sql @@ -0,0 +1 @@ +select * from lineitem where l_shipdate between date '1996-01-1' and date '1997-12-31'; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenNumbers.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenNumbers.sql new file mode 100644 index 0000000000..e3896659e7 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testBetweenNumbers.sql @@ -0,0 +1 @@ +select * from lineitem where l_orderkey between 1 and 2; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCaseWhenFilter.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCaseWhenFilter.sql new file mode 100644 index 0000000000..c2f903db43 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCaseWhenFilter.sql @@ -0,0 +1,6 @@ +select * from lineitem where + case + when l_returnflag = 'N' then true + else false + end +; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCountAsterisk.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCountAsterisk.sql new file mode 100644 index 0000000000..5c69fb83b4 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCountAsterisk.sql @@ -0,0 +1 @@ +select count(*) from lineitem; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCtasToHdfs.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCtasToHdfs.sql new file mode 100644 index 0000000000..75d051619e --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testCtasToHdfs.sql @@ -0,0 +1 @@ +SELECT * FROM mytmp.offload; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testDateTimeTypes.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testDateTimeTypes.sql new file mode 100644 index 0000000000..035c97afde --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testDateTimeTypes.sql @@ -0,0 +1 @@ +SELECT * FROM "DATETIME_TYPES" WHERE date_field > DATE '1980-01-01'; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFixedLengthFields.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFixedLengthFields.sql new file mode 100644 index 0000000000..d68b9fc416 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFixedLengthFields.sql @@ -0,0 +1 @@ +select l_shipinstruct, l_shipmode from lineitem where l_orderkey > 1 and l_linenumber > 0; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFunctionWithinFilter.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFunctionWithinFilter.sql new file mode 100644 index 0000000000..204f28bb42 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testFunctionWithinFilter.sql @@ -0,0 +1 @@ +select * from lineitem where concat(l_returnflag, l_linestatus) = 'NO' \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithLiterals.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithLiterals.sql new file mode 100644 index 0000000000..63a366f1a3 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithLiterals.sql @@ -0,0 +1 @@ +select * from lineitem where l_returnflag in ('N'); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithNumbers.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithNumbers.sql new file mode 100644 index 0000000000..858208f436 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testInPredicateWithNumbers.sql @@ -0,0 +1 @@ +select * from lineitem where l_orderkey in (1, 2); \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testProjectedColumns.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testProjectedColumns.sql new file mode 100644 index 0000000000..9d5eb58072 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testProjectedColumns.sql @@ -0,0 +1 @@ +select l_orderkey, l_shipdate from lineitem where l_orderkey > 1 and l_linenumber > 0; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testQueryWithConnProperties.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testQueryWithConnProperties.sql new file mode 100644 index 0000000000..c9495d658c --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testQueryWithConnProperties.sql @@ -0,0 +1 @@ +select * from lineitem where l_orderkey > 1; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testSimpleFilter.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testSimpleFilter.sql new file mode 100644 index 0000000000..c9495d658c --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testSimpleFilter.sql @@ -0,0 +1 @@ +select * from lineitem where l_orderkey > 1; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q1.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q1.sql new file mode 100644 index 0000000000..647e1529f5 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q1.sql @@ -0,0 +1,21 @@ +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice*(1-l_discount)) as sum_disc_price, + sum(l_extendedprice*(1-l_discount)*(1+l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-09-01' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part.sql new file mode 100644 index 0000000000..ffaaf87a91 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part.sql @@ -0,0 +1,18 @@ +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment, + ps_supplycost, + r_name, + p_type, + p_size +from + region join nation on n_regionkey = r_regionkey and r_name = 'AMERICA' + join supplier on s_nationkey = n_nationkey + join partsupp on s_suppkey = ps_suppkey + join part on p_partkey = ps_partkey and p_type like '%BRASS' and p_size = 15; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.sql new file mode 100644 index 0000000000..921fce765f --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.sql @@ -0,0 +1,18 @@ +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment, + ps_supplycost, + r_name, + p_type, + p_size +from + default.region join tpch.nation on n_regionkey = r_regionkey and r_name = 'AMERICA' + join default.supplier on s_nationkey = n_nationkey + join default.partsupp on s_suppkey = ps_suppkey + join default.part on p_partkey = ps_partkey and p_type like '%BRASS' and p_size = 15; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q3.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q3.sql new file mode 100644 index 0000000000..7d338dd8b9 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q3.sql @@ -0,0 +1,22 @@ +select + l_orderkey, + sum(l_extendedprice*(1-l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment in ('BUILDING', 'AUTOMOBILE', 'MACHINERY', 'HOUSEHOLD') -- modified for selectivity + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1996-12-31' -- modified for selectivity + and l_shipdate > date '1992-03-15' -- modified for selectivity +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q5.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q5.sql new file mode 100644 index 0000000000..d3c84ad7b3 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testTPCH_Q5.sql @@ -0,0 +1,20 @@ +select n_name, revenue::float4 as revenue from ( +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey and + r_name = 'ASIA' and + o_orderdate >= date '1993-01-01' and o_orderdate < date '1997-01-01' +group by + n_name +order by + revenue desc +) w; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testVariableLengthFields.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testVariableLengthFields.sql new file mode 100644 index 0000000000..b53b235e55 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLQueryTests/testVariableLengthFields.sql @@ -0,0 +1 @@ +select l_comment from lineitem where l_orderkey > 1 and l_linenumber > 0; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectAll.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectAll.sql new file mode 100644 index 0000000000..cda515b8eb --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectAll.sql @@ -0,0 +1 @@ +select * from lineitem; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectLimit.sql b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectLimit.sql new file mode 100644 index 0000000000..79cf804b9a --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/queries/TestMySQLSimpleQueryTests/testSelectLimit.sql @@ -0,0 +1 @@ +select * from lineitem limit 3; \ No newline at end of file diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenDates.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenDates.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenDates.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenNumbers.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenNumbers.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testBetweenNumbers.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCaseWhenFilter.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCaseWhenFilter.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCaseWhenFilter.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCountAsterisk.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCountAsterisk.result new file mode 100644 index 0000000000..70448010bd --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCountAsterisk.result @@ -0,0 +1,3 @@ +?count +------------------------------- +5 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCtasToHdfs.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCtasToHdfs.result new file mode 100644 index 0000000000..804338ca36 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testCtasToHdfs.result @@ -0,0 +1,7 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a +3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco +3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL,unusual accounts. eve diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testDateTimeTypes.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testDateTimeTypes.result new file mode 100644 index 0000000000..cd68a388ff --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testDateTimeTypes.result @@ -0,0 +1,6 @@ +date_field,time_field,ts_field +------------------------------- +1980-04-01,01:50:27,1980-04-01 01:50:27 +1984-01-05,02:45:31,1984-01-05 02:45:31 +1984-01-15,14:12:58,1984-01-15 14:12:58 +2015-03-21,08:11:01,2015-03-21 08:11:01 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFixedLengthFields.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFixedLengthFields.result new file mode 100644 index 0000000000..47085e22dc --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFixedLengthFields.result @@ -0,0 +1,5 @@ +l_shipinstruct,l_shipmode +------------------------------- +NONE,AIR +TAKE BACK RETURN,RAIL +TAKE BACK RETURN,RAIL diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFunctionWithinFilter.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFunctionWithinFilter.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testFunctionWithinFilter.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithLiterals.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithLiterals.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithLiterals.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithNumbers.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithNumbers.result new file mode 100644 index 0000000000..2f6e7a145b --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testInPredicateWithNumbers.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testProjectedColumns.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testProjectedColumns.result new file mode 100644 index 0000000000..e858d4c380 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testProjectedColumns.result @@ -0,0 +1,5 @@ +l_orderkey,l_shipdate +------------------------------- +2,1997-01-28 +3,1993-11-09 +3,1994-02-02 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testQueryWithConnProperties.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testQueryWithConnProperties.result new file mode 100644 index 0000000000..2eb253efbd --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testQueryWithConnProperties.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a +3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco +3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL,unusual accounts. eve diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testSimpleFilter.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testSimpleFilter.result new file mode 100644 index 0000000000..2eb253efbd --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testSimpleFilter.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a +3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco +3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL,unusual accounts. eve diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q1.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q1.result new file mode 100644 index 0000000000..3020f561dc --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q1.result @@ -0,0 +1,4 @@ +l_returnflag,l_linestatus,sum_qty,sum_base_price,sum_disc_price,sum_charge,avg_qty,avg_price,avg_disc,count_order +------------------------------- +N,O,91.0,111845.85,106860.63639999999,112012.46995200001,30.333333333333332,37281.950000000004,0.043333333333333335,3 +R,F,94.0,100854.52,92931.39000000001,92931.39000000001,47.0,50427.26,0.08,2 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part.result new file mode 100644 index 0000000000..64d8e12272 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part.result @@ -0,0 +1,3 @@ +s_acctbal,s_name,n_name,p_partkey,p_mfgr,s_address,s_phone,s_comment,ps_supplycost,r_name,p_type,p_size +------------------------------- +4192.4,Supplier#000000003,ARGENTINA,2,Manufacturer#1,q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3,11-383-516-1199,blithely silent requests after the express dependencies are sl,1.01,AMERICA,LARGE BRUSHED BRASS,15 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.result new file mode 100644 index 0000000000..64d8e12272 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q2_Part_MixedStorage.result @@ -0,0 +1,3 @@ +s_acctbal,s_name,n_name,p_partkey,p_mfgr,s_address,s_phone,s_comment,ps_supplycost,r_name,p_type,p_size +------------------------------- +4192.4,Supplier#000000003,ARGENTINA,2,Manufacturer#1,q1,G3Pj6OjIuUYfUoH18BFTKP5aU9bEV3,11-383-516-1199,blithely silent requests after the express dependencies are sl,1.01,AMERICA,LARGE BRUSHED BRASS,15 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q3.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q3.result new file mode 100644 index 0000000000..386b650cd5 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q3.result @@ -0,0 +1,5 @@ +l_orderkey,revenue,o_orderdate,o_shippriority +------------------------------- +1,62166.1764,1996-01-02,0 +2,44694.46,1996-12-01,0 +3,92931.39000000001,1993-10-14,0 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q5.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q5.result new file mode 100644 index 0000000000..f249da0198 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testTPCH_Q5.result @@ -0,0 +1,27 @@ +n_name,revenue +------------------------------- +ALGERIA,1798128.2 +ARGENTINA,1798128.2 +BRAZIL,1798128.2 +CANADA,1798128.2 +CHINA,1798128.2 +EGYPT,1798128.2 +ETHIOPIA,1798128.2 +FRANCE,1798128.2 +GERMANY,1798128.2 +INDIA,1798128.2 +INDONESIA,1798128.2 +IRAN,1798128.2 +IRAQ,1798128.2 +JAPAN,1798128.2 +JORDAN,1798128.2 +KENYA,1798128.2 +MOROCCO,1798128.2 +MOZAMBIQUE,1798128.2 +PERU,1798128.2 +ROMANIA,1798128.2 +RUSSIA,1798128.2 +SAUDI ARABIA,1798128.2 +UNITED KINGDOM,1798128.2 +UNITED STATES,1798128.2 +VIETNAM,1798128.2 diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testVariableLengthFields.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testVariableLengthFields.result new file mode 100644 index 0000000000..b354aa9570 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLQueryTests/testVariableLengthFields.result @@ -0,0 +1,5 @@ +l_comment +------------------------------- +ongside of the furiously brave acco +unusual accounts. eve +ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectAll.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectAll.result new file mode 100644 index 0000000000..dd34c549df --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectAll.result @@ -0,0 +1,7 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a +3,2,1798,1,45.0,54058.05,0.06,0.0,R,F,1994-02-02,1994-01-04,1994-02-23,NONE,AIR,ongside of the furiously brave acco +3,3,6540,2,49.0,46796.47,0.1,0.0,R,F,1993-11-09,1993-12-20,1993-11-24,TAKE BACK RETURN,RAIL,unusual accounts. eve diff --git a/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectLimit.result b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectLimit.result new file mode 100644 index 0000000000..a7f7d20f34 --- /dev/null +++ b/tajo-storage/tajo-storage-mysql/src/test/resources/results/TestMySQLSimpleQueryTests/testSelectLimit.result @@ -0,0 +1,5 @@ +l_orderkey,l_partkey,l_suppkey,l_linenumber,l_quantity,l_extendedprice,l_discount,l_tax,l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate,l_shipinstruct,l_shipmode,l_comment +------------------------------- +1,1,7706,1,17.0,21168.23,0.04,0.02,N,O,1996-03-13,1996-02-12,1996-03-22,DELIVER IN PERSON,TRUCK,egular courts above the +1,1,7311,2,36.0,45983.16,0.09,0.06,N,O,1996-04-12,1996-02-28,1996-04-20,TAKE BACK RETURN,MAIL,ly final dependencies: slyly bold +2,2,1191,1,38.0,44694.46,0.0,0.05,N,O,1997-01-28,1997-01-14,1997-02-02,TAKE BACK RETURN,RAIL,ven requests. deposits breach a diff --git a/tajo-storage/tajo-storage-pgsql/src/main/java/org/apache/tajo/storage/pgsql/PgSQLTablespace.java b/tajo-storage/tajo-storage-pgsql/src/main/java/org/apache/tajo/storage/pgsql/PgSQLTablespace.java index 8386f74995..90c52011c8 100644 --- a/tajo-storage/tajo-storage-pgsql/src/main/java/org/apache/tajo/storage/pgsql/PgSQLTablespace.java +++ b/tajo-storage/tajo-storage-pgsql/src/main/java/org/apache/tajo/storage/pgsql/PgSQLTablespace.java @@ -24,6 +24,7 @@ import org.apache.tajo.exception.TajoRuntimeException; import org.apache.tajo.exception.UndefinedTableException; import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.NullScanner; import org.apache.tajo.storage.Scanner; import org.apache.tajo.storage.fragment.Fragment; @@ -82,7 +83,7 @@ public int hashCode() { @Override public long getTableVolume(TableDesc table, Optional filter) { - String sql = "SELECT pg_table_size('" + CatalogUtil.extractSimpleName(table.getName()) + "')"; + String sql = "SELECT pg_table_size('" + IdentifierUtil.extractSimpleName(table.getName()) + "')"; try (Statement stmt = conn.createStatement(); ResultSet rs = stmt.executeQuery(sql)) { diff --git a/tajo-storage/tajo-storage-pgsql/src/test/java/org/apache/tajo/storage/pgsql/TestPgSQLJdbcTableSpace.java b/tajo-storage/tajo-storage-pgsql/src/test/java/org/apache/tajo/storage/pgsql/TestPgSQLJdbcTableSpace.java index 0d0e15ec26..875d70ee5d 100644 --- a/tajo-storage/tajo-storage-pgsql/src/test/java/org/apache/tajo/storage/pgsql/TestPgSQLJdbcTableSpace.java +++ b/tajo-storage/tajo-storage-pgsql/src/test/java/org/apache/tajo/storage/pgsql/TestPgSQLJdbcTableSpace.java @@ -74,7 +74,7 @@ public void testGetSplits() throws IOException, TajoException { Tablespace space = TablespaceManager.getByName("pgsql_cluster"); MetadataProvider provider = space.getMetadataProvider(); TableDesc table = provider.getTableDesc(null, "lineitem"); - List fragments = space.getSplits("lineitem", table, null); + List fragments = space.getSplits("lineitem", table, false, null); assertNotNull(fragments); assertEquals(1, fragments.size()); } diff --git a/tajo-tablespace-example/pom.xml b/tajo-tablespace-example/pom.xml new file mode 100644 index 0000000000..7e15e0c14f --- /dev/null +++ b/tajo-tablespace-example/pom.xml @@ -0,0 +1,256 @@ + + + + + + tajo-project + org.apache.tajo + 0.12.0-SNAPSHOT + ../tajo-project/pom.xml + + 4.0.0 + + tajo-tablespace-example + jar + Tajo Tablespace Example + + UTF-8 + UTF-8 + + + + + + org.apache.rat + apache-rat-plugin + + + derby.log + src/test/resources/dataset/** + src/test/resources/queries/** + src/test/resources/results/** + + + + + verify + + check + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + true + + -Xms128m -Xmx1024m -Dfile.encoding=UTF-8 + + + + org.apache.maven.plugins + maven-antrun-plugin + + + create-protobuf-generated-sources-directory + initialize + + + + + + + run + + + + + + org.codehaus.mojo + exec-maven-plugin + 1.2 + + + generate-sources + generate-sources + + protoc + + -Isrc/main/proto/ + --proto_path=../tajo-common/src/main/proto + --proto_path=../tajo-catalog/tajo-catalog-common/src/main/proto + --java_out=target/generated-sources/proto + src/main/proto/ExampleHttpFragmentProtos.proto + + + + exec + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.5 + + + add-source + generate-sources + + add-source + + + + target/generated-sources/proto + + + + + + + + + + + org.apache.tajo + tajo-common + provided + + + org.apache.tajo + tajo-catalog-common + provided + + + org.apache.tajo + tajo-plan + provided + + + org.apache.tajo + tajo-storage-common + provided + + + org.apache.tajo + tajo-storage-hdfs + provided + + + org.apache.tajo + tajo-rpc-common + + + org.apache.tajo + tajo-cluster-tests + test-jar + test + + + junit + junit + test + + + io.netty + netty-transport + test + + + io.netty + netty-codec-http + test + + + + org.apache.hadoop + hadoop-common + provided + + + org.apache.hadoop + hadoop-minicluster + test + + + commons-el + commons-el + + + tomcat + jasper-runtime + + + tomcat + jasper-compiler + + + org.mortbay.jetty + jsp-2.1-jetty + + + com.sun.jersey.jersey-test-framework + jersey-test-framework-grizzly2 + + + netty-all + io.netty + + + + + org.apache.hadoop + hadoop-hdfs + test + + + commons-el + commons-el + + + tomcat + jasper-runtime + + + tomcat + jasper-compiler + + + org.mortbay.jetty + jsp-2.1-jetty + + + com.sun.jersey.jersey-test-framework + jersey-test-framework-grizzly2 + + + netty-all + io.netty + + + + + + \ No newline at end of file diff --git a/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragment.java b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragment.java new file mode 100644 index 0000000000..86ebc10257 --- /dev/null +++ b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragment.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import org.apache.tajo.storage.fragment.AbstractFileFragment; +import org.apache.tajo.storage.fragment.BuiltinFragmentKinds; + +import java.net.URI; + +public class ExampleHttpFileFragment extends AbstractFileFragment { + + public ExampleHttpFileFragment(URI uri, + String inputSourceId, + long startKey, + long endKey) { + super(BuiltinFragmentKinds.HTTP, uri, inputSourceId, startKey, endKey, endKey - startKey, null); + } +} diff --git a/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragmentSerde.java b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragmentSerde.java new file mode 100644 index 0000000000..866526922d --- /dev/null +++ b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileFragmentSerde.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import com.google.protobuf.GeneratedMessage.Builder; +import org.apache.tajo.storage.fragment.FragmentSerde; +import org.apache.tajo.storage.http.ExampleHttpFragmentProtos.ExampleHttpFileFragmentProto; + +import java.net.URI; + +public class ExampleHttpFileFragmentSerde + implements FragmentSerde { + + @Override + public Builder newBuilder() { + return ExampleHttpFileFragmentProto.newBuilder(); + } + + @Override + public ExampleHttpFileFragmentProto serialize(ExampleHttpFileFragment fragment) { + return ExampleHttpFileFragmentProto.newBuilder() + .setUri(fragment.getUri().toASCIIString()) + .setTableName(fragment.getInputSourceId()) + .setStartKey(fragment.getStartKey()) + .setEndKey(fragment.getEndKey()) + .build(); + } + + @Override + public ExampleHttpFileFragment deserialize(ExampleHttpFileFragmentProto proto) { + return new ExampleHttpFileFragment( + URI.create(proto.getUri()), + proto.getTableName(), + proto.getStartKey(), + proto.getEndKey() + ); + } +} diff --git a/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileTablespace.java b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileTablespace.java new file mode 100644 index 0000000000..258d505952 --- /dev/null +++ b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpFileTablespace.java @@ -0,0 +1,231 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import com.google.common.collect.Lists; +import net.minidev.json.JSONObject; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.BuiltinStorages; +import org.apache.tajo.ExecutionBlockId; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.SortSpec; +import org.apache.tajo.catalog.TableDesc; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.exception.TajoException; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.plan.LogicalPlan; +import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.plan.logical.LogicalNode; +import org.apache.tajo.storage.FormatProperty; +import org.apache.tajo.storage.StorageProperty; +import org.apache.tajo.storage.Tablespace; +import org.apache.tajo.storage.TupleRange; +import org.apache.tajo.storage.fragment.Fragment; +import org.jboss.netty.handler.codec.http.HttpHeaders.Names; + +import javax.annotation.Nullable; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.util.List; +import java.util.Optional; + +/** + * Example read-only tablespace for HTTP protocol. + * + * An example table can be created by using the following SQL query. + * + * CREATE TABLE http_test (*) TABLESPACE http_example USING ex_http_json WITH ('path'='2015-01-01-15.json.gz', + * 'compression.codec'='org.apache.hadoop.io.compress.GzipCodec’); + */ +public class ExampleHttpFileTablespace extends Tablespace { + private static final Log LOG = LogFactory.getLog(ExampleHttpFileTablespace.class); + + static final String PATH = "path"; + + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Tablespace properties + ////////////////////////////////////////////////////////////////////////////////////////////////// + private static final StorageProperty STORAGE_PROPERTY = + new StorageProperty( + BuiltinStorages.JSON, // default format is json + false, // is not movable + false, // is not writable + true, // allow arbitrary path + false // doesn't provide metadata + ); + + private static final FormatProperty FORMAT_PROPERTY = + new FormatProperty( + false, // doesn't support insert + false, // doesn't support direct insert + false // doesn't support result staging + ); + + public ExampleHttpFileTablespace(String name, URI uri, JSONObject config) { + super(name, uri, config); + + LOG.info("ExampleHttpFileTablespace is initialized for " + uri); + } + + @Override + protected void storageInit() throws IOException { + // Add initialization code for your tablespace + } + + @Override + public long getTableVolume(TableDesc table, Optional notUsed) { + HttpURLConnection connection = null; + + try { + connection = (HttpURLConnection) new URL(table.getUri().toASCIIString()).openConnection(); + connection.setRequestMethod("HEAD"); + connection.connect(); + return connection.getHeaderFieldLong(Names.CONTENT_LENGTH, -1); + + } catch (IOException e) { + throw new TajoInternalError(e); + + } finally { + if (connection != null) { + connection.disconnect(); + } + } + } + + @Override + public URI getRootUri() { + return uri; + } + + @Override + public URI getTableUri(TableMeta meta, String databaseName, String tableName) { + String tablespaceUriString = uri.toASCIIString(); + String tablePath = meta.getProperty(PATH); + + if (!tablespaceUriString.endsWith("/") && !tablePath.startsWith("/")) { + tablePath = "/" + tablePath; + } + + return URI.create(tablespaceUriString + tablePath); + } + + @Override + public List getSplits(String inputSourceId, + TableDesc tableDesc, + boolean requireSort, + @Nullable EvalNode filterCondition) + throws IOException, TajoException { + + // getSplits() should return multiple fragments for distributed processing of a large data. + // This example tablespace returns only one fragment for the whole data for simplicity, + // but this may significantly increase the query processing time. + + long tableVolume = getTableVolume(tableDesc, Optional.empty()); + return Lists.newArrayList(new ExampleHttpFileFragment(tableDesc.getUri(), inputSourceId, 0, tableVolume)); + } + + @Override + public StorageProperty getProperty() { + return STORAGE_PROPERTY; + } + + @Override + public FormatProperty getFormatProperty(TableMeta meta) { + return FORMAT_PROPERTY; + } + + @Override + public void close() { + // do nothing + } + + @Override + public TupleRange[] getInsertSortRanges(OverridableConf queryContext, + TableDesc tableDesc, + Schema inputSchema, + SortSpec[] sortSpecs, + TupleRange dataRange) throws IOException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public void verifySchemaToWrite(TableDesc tableDesc, Schema outSchema) throws TajoException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public void createTable(TableDesc table, boolean ifNotExists) throws TajoException, IOException { + HttpURLConnection connection = null; + + try { + connection = (HttpURLConnection) new URL(table.getUri().toASCIIString()).openConnection(); + connection.setRequestMethod("HEAD"); + connection.connect(); + + if (connection.getResponseCode() == 404) { + throw new FileNotFoundException(); + } + + } catch (IOException e) { + throw new TajoInternalError(e); + + } finally { + if (connection != null) { + connection.disconnect(); + } + } + } + + @Override + public void purgeTable(TableDesc tableDesc) throws IOException, TajoException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public void prepareTable(LogicalNode node) throws IOException, TajoException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public Path commitTable(OverridableConf queryContext, + ExecutionBlockId finalEbId, + LogicalPlan plan, + Schema schema, + TableDesc tableDesc) throws IOException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public void rollbackTable(LogicalNode node) throws IOException, TajoException { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public URI getStagingUri(OverridableConf context, String queryId, TableMeta meta) throws IOException { + throw new TajoRuntimeException(new UnsupportedException()); + } +} diff --git a/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonLineReader.java b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonLineReader.java new file mode 100644 index 0000000000..61e4d6c377 --- /dev/null +++ b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonLineReader.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.hadoop.io.compress.Decompressor; +import org.apache.hadoop.io.compress.SplittableCompressionCodec; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.BufferPool; +import org.apache.tajo.storage.ByteBufInputChannel; +import org.apache.tajo.storage.compress.CodecPool; +import org.apache.tajo.storage.fragment.AbstractFileFragment; +import org.apache.tajo.storage.text.ByteBufLineReader; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.concurrent.atomic.AtomicInteger; + +public class ExampleHttpJsonLineReader implements Closeable { + + private final static Log LOG = LogFactory.getLog(ExampleHttpJsonLineReader.class); + + private Configuration conf; + + private HttpURLConnection connection; + private InputStream is; + + private CompressionCodec codec; + private Decompressor decompressor; + + private long startOffset, endOffset, pos; + private boolean eof = true; + private ByteBufLineReader lineReader; + private AtomicInteger lineReadBytes = new AtomicInteger(); + private ExampleHttpFileFragment fragment; + private final int bufferSize; + + public ExampleHttpJsonLineReader(Configuration conf, + AbstractFileFragment fragment, + int bufferSize) { + this.conf = conf; + this.fragment = (ExampleHttpFileFragment) fragment; + this.bufferSize = bufferSize; + + CompressionCodecFactory factory = new CompressionCodecFactory(conf); + codec = factory.getCodec(fragment.getPath()); + if (this.codec instanceof SplittableCompressionCodec) { + // bzip2 does not support multi-thread model + throw new TajoRuntimeException(new UnsupportedException(codec.getDefaultExtension())); + } + } + + public void init() throws IOException { + if (connection != null) { + throw new IOException(this.getClass() + " is already initialized"); + } + + pos = startOffset = fragment.getStartKey(); + endOffset = fragment.getEndKey(); + + URL url = new URL(fragment.getUri().toASCIIString()); + connection = (HttpURLConnection) url.openConnection(); + + is = connection.getInputStream(); + + ByteBuf buf; + if (codec != null) { + decompressor = CodecPool.getDecompressor(codec); + is = codec.createInputStream(is, decompressor); + + buf = BufferPool.directBuffer(bufferSize); + + } else { + buf = BufferPool.directBuffer((int) Math.min(bufferSize, fragment.getLength())); + + } + + lineReader = new ByteBufLineReader(new ByteBufInputChannel(is), buf); + + eof = false; + } + + public ByteBuf readLine() throws IOException { + if (eof) { + return null; + } + + ByteBuf buf = lineReader.readLineBuf(lineReadBytes); + pos += lineReadBytes.get(); + if (buf == null) { + eof = true; + } + + return buf; + } + + public boolean isCompressed() { + return codec != null; + } + + public long getPos() { + return pos; + } + + public long getReadBytes() { + return pos - startOffset; + } + + public boolean isEof() { + return eof; + } + + @Override + public void close() throws IOException { + try { + IOUtils.cleanup(LOG, lineReader); + + if (connection != null) { + connection.disconnect(); + } + + is = null; + lineReader = null; + + } finally { + CodecPool.returnDecompressor(decompressor); + decompressor = null; + } + } +} diff --git a/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonScanner.java b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonScanner.java new file mode 100644 index 0000000000..6c41a50abe --- /dev/null +++ b/tajo-tablespace-example/src/main/java/org/apache/tajo/storage/http/ExampleHttpJsonScanner.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.IOUtils; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.statistics.TableStats; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.storage.EmptyTuple; +import org.apache.tajo.storage.FileScanner; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; +import org.apache.tajo.storage.fragment.Fragment; +import org.apache.tajo.storage.json.JsonLineDeserializer; +import org.apache.tajo.storage.text.TextLineParsingError; +import org.apache.tajo.unit.StorageUnit; + +import java.io.IOException; + +import static org.apache.tajo.storage.StorageConstants.DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM; +import static org.apache.tajo.storage.StorageConstants.TEXT_ERROR_TOLERANCE_MAXNUM; +import static org.apache.tajo.storage.text.DelimitedTextFile.READ_BUFFER_SIZE; + +public class ExampleHttpJsonScanner extends FileScanner { + + private static final Log LOG = LogFactory.getLog(ExampleHttpJsonScanner.class); + + private VTuple outTuple; + + private long limit; + + private final long startOffset; + private final long endOffset; + + private ExampleHttpJsonLineReader reader; + private JsonLineDeserializer deserializer; + + private int errorPrintOutMaxNum = 5; + /** Maximum number of permissible errors */ + private final int maxAllowedErrorCount; + /** How many errors have occurred? */ + private int errorNum; + + private boolean splittable = false; + + private long recordCount = 0; + + public ExampleHttpJsonScanner(Configuration conf, Schema schema, TableMeta tableMeta, Fragment fragment) + throws IOException { + super(conf, schema, tableMeta, fragment); + + reader = new ExampleHttpJsonLineReader(conf, this.fragment, conf.getInt(READ_BUFFER_SIZE, 128 * StorageUnit.KB)); + if (!this.reader.isCompressed()) { + splittable = true; + } + + startOffset = this.fragment.getStartKey(); + endOffset = this.fragment.getEndKey(); + + maxAllowedErrorCount = + Integer.parseInt(tableMeta.getProperty(TEXT_ERROR_TOLERANCE_MAXNUM, DEFAULT_TEXT_ERROR_TOLERANCE_MAXNUM)); + } + + @Override + public void init() throws IOException { + + reader.init(); + + if (targets == null) { + targets = schema.toArray(); + } + + reset(); + + super.init(); + } + + @Override + public Tuple next() throws IOException { + + if (reader.isEof()) { + return null; // Indicate to the parent operator that there is no more data. + } + + // Read lines until it reads a valid tuple or EOS (end of stream). + while (!reader.isEof()) { + + ByteBuf buf = reader.readLine(); + + if (buf == null) { // The null buf means that there is no more lines. + return null; + } + + // When the number of projection columns is 0, the read line doesn't have to be parsed. + if (targets.length == 0) { + recordCount++; + return EmptyTuple.get(); + } + + try { + deserializer.deserialize(buf, outTuple); + + // Once a line is normally parsed, exits the while loop. + break; + + } catch (TextLineParsingError tlpe) { + + errorNum++; + + // The below line may print too many logs. + LOG.warn("Ignore Text Parse Error (" + errorNum + "): ", tlpe); + + // If the number of found errors exceeds the configured tolerable error count, + // throw the error. + if (maxAllowedErrorCount >= 0 && errorNum > maxAllowedErrorCount) { + throw new IOException(tlpe); + } + } + } + + recordCount++; + + return outTuple; + } + + @Override + public void reset() throws IOException { + recordCount = 0; + + if (reader.getReadBytes() > 0) { + reader.close(); + + reader = new ExampleHttpJsonLineReader(conf, fragment, conf.getInt(READ_BUFFER_SIZE, 128 * StorageUnit.KB)); + reader.init(); + } + + if(deserializer != null) { + deserializer.release(); + } + + deserializer = new JsonLineDeserializer(schema, meta, targets); + deserializer.init(); + + outTuple = new VTuple(targets.length); + + // skip first line if it reads from middle of file + if (startOffset > 0) { + reader.readLine(); + } + } + + @Override + public void close() throws IOException { + try { + + if (deserializer != null) { + deserializer.release(); + } + + if (reader != null) { + inputStats.setReadBytes(reader.getReadBytes()); + inputStats.setNumRows(recordCount); + } + + } finally { + IOUtils.cleanup(LOG, reader); + outTuple = null; + } + } + + @Override + public boolean isProjectable() { + return true; + } + + @Override + public void setTarget(Column[] targets) { + this.targets = targets; + } + + @Override + public boolean isSelectable() { + return false; + } + + @Override + public void setFilter(EvalNode filter) { + throw new TajoRuntimeException(new UnsupportedException()); + } + + @Override + public void setLimit(long num) { + this.limit = num; + } + + @Override + public boolean isSplittable() { + return splittable; + } + + @Override + public float getProgress() { + if(!inited) return super.getProgress(); + + if (reader.isEof()) { // if the reader reaches EOS + return 1.0f; + } + + long currentPos = reader.getPos(); + long readBytes = currentPos - startOffset; + long remainingBytes = Math.max(endOffset - currentPos, 0); + return Math.min(1.0f, (float) (readBytes) / (float) (readBytes + remainingBytes)); + } + + @Override + public TableStats getInputStats() { + if (inputStats != null && reader != null) { + inputStats.setReadBytes(reader.getReadBytes()); //Actual Processed Bytes. (decompressed bytes + overhead) + inputStats.setNumRows(recordCount); + inputStats.setNumBytes(fragment.getLength()); + } + + return inputStats; + } +} diff --git a/tajo-tablespace-example/src/main/proto/ExampleHttpFragmentProtos.proto b/tajo-tablespace-example/src/main/proto/ExampleHttpFragmentProtos.proto new file mode 100644 index 0000000000..586e46451d --- /dev/null +++ b/tajo-tablespace-example/src/main/proto/ExampleHttpFragmentProtos.proto @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +option java_package = "org.apache.tajo.storage.http"; +option java_outer_classname = "ExampleHttpFragmentProtos"; +option optimize_for = SPEED; +option java_generic_services = false; +option java_generate_equals_and_hash = true; + +import "CatalogProtos.proto"; + +message ExampleHttpFileFragmentProto { + required string uri = 1; + required string table_name = 2; + required int64 start_key = 3; + required int64 end_key = 4; +} \ No newline at end of file diff --git a/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerHandler.java b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerHandler.java new file mode 100644 index 0000000000..fd25da8975 --- /dev/null +++ b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerHandler.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import io.netty.buffer.Unpooled; +import io.netty.channel.*; +import io.netty.handler.codec.http.*; +import io.netty.handler.codec.http.HttpHeaders.Names; +import io.netty.util.CharsetUtil; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import javax.activation.MimetypesFileTypeMap; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; + +import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.OK; +import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; + +public class ExampleHttpServerHandler extends SimpleChannelInboundHandler { + + private static final Log LOG = LogFactory.getLog(ExampleHttpServerHandler.class); + + @Override + protected void channelRead0(ChannelHandlerContext context, FullHttpRequest request) throws Exception { + + if (request.getMethod().equals(HttpMethod.HEAD)) { + + processHead(context, request); + + } else if (request.getMethod().equals(HttpMethod.GET)) { + + processGet(context, request); + + } else { + // error + String msg = "Not supported method: " + request.getMethod(); + LOG.error(msg); + context.writeAndFlush(getBadRequest(msg)); + } + } + + private void processHead(ChannelHandlerContext context, FullHttpRequest request) { + HttpHeaders headers = request.headers(); + FullHttpResponse response = null; + + if (headers.contains(Names.CONTENT_LENGTH)) { + + try { + File file = getRequestedFile(request.getUri()); + + response = new DefaultFullHttpResponse( + HTTP_1_1, + request.getDecoderResult().isSuccess() ? OK : BAD_REQUEST + ); + + HttpHeaders.setContentLength(response, file.length()); + + + } catch (FileNotFoundException | URISyntaxException e) { + response = getBadRequest(e.getMessage()); + } + } + + context.writeAndFlush(response); + } + + private void processGet(ChannelHandlerContext context, FullHttpRequest request) { + try { + File file = getRequestedFile(request.getUri()); + + RandomAccessFile raf = new RandomAccessFile(file, "r"); + long fileLength = raf.length(); + + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + HttpHeaders.setContentLength(response, fileLength); + setContentTypeHeader(response, file); + + context.write(response); + + context.write(new DefaultFileRegion(raf.getChannel(), 0, fileLength)); + + // Write the end marker. + ChannelFuture future = context.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); + future.addListener(ChannelFutureListener.CLOSE); + + } catch (IOException | URISyntaxException e) { + context.writeAndFlush(getBadRequest(e.getMessage())); + } + } + + private static File getRequestedFile(String uri) throws FileNotFoundException, URISyntaxException { + String path = URI.create(uri).getPath(); + URL url = ClassLoader.getSystemResource("dataset/" + path); + + if (url == null) { + throw new FileNotFoundException(uri); + } + return new File(url.toURI()); + } + + private static FullHttpResponse getBadRequest(String message) { + return new DefaultFullHttpResponse(HTTP_1_1, BAD_REQUEST, + Unpooled.copiedBuffer(message, CharsetUtil.UTF_8)); + } + + @Override + public void exceptionCaught(ChannelHandlerContext context, Throwable cause) { + LOG.error(cause.getMessage(), cause); + if (context.channel().isOpen()) { + context.channel().close(); + } + } + + /** + * Sets the content type header for the HTTP Response + * @param response HTTP response + * @param file file to extract content type + */ + private static void setContentTypeHeader(HttpResponse response, File file) { + MimetypesFileTypeMap mimeTypesMap = new MimetypesFileTypeMap(); + response.headers().set(CONTENT_TYPE, mimeTypesMap.getContentType(file.getPath())); + } +} diff --git a/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerInitializer.java b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerInitializer.java new file mode 100644 index 0000000000..2cd6340ade --- /dev/null +++ b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpServerInitializer.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpServerCodec; +import io.netty.handler.stream.ChunkedWriteHandler; + +public class ExampleHttpServerInitializer extends ChannelInitializer { + + @Override + protected void initChannel(SocketChannel socketChannel) throws Exception { + ChannelPipeline pipeline = socketChannel.pipeline(); + + pipeline.addLast(new HttpServerCodec()); + pipeline.addLast(new HttpObjectAggregator(65536)); + pipeline.addLast(new ChunkedWriteHandler()); + pipeline.addLast(new ExampleHttpServerHandler()); + } +} diff --git a/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpTablespaceTestServer.java b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpTablespaceTestServer.java new file mode 100644 index 0000000000..aa7acc85d8 --- /dev/null +++ b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/ExampleHttpTablespaceTestServer.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.io.Closeable; +import java.net.InetSocketAddress; + +public class ExampleHttpTablespaceTestServer implements Closeable { + + private final static Log LOG = LogFactory.getLog(ExampleHttpTablespaceTestServer.class); + + private ServerBootstrap bootstrap; + private Channel channel; + + public void init() throws InterruptedException { + EventLoopGroup group = new NioEventLoopGroup(1); + + bootstrap = new ServerBootstrap(); + bootstrap.group(group) + .channel(NioServerSocketChannel.class) + .childHandler(new ExampleHttpServerInitializer()); + + channel = bootstrap.bind(0).sync().channel(); + + LOG.info(ExampleHttpTablespaceTestServer.class.getSimpleName() + " listening on port " + getAddress().getPort()); + } + + public InetSocketAddress getAddress() { + return (InetSocketAddress) channel.localAddress(); + } + + public void close() { + if (bootstrap != null) { + if (bootstrap.group() != null) { + bootstrap.group().shutdownGracefully(); + } + } + + if (channel != null) { + channel.close(); + } + } +} diff --git a/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/TestExampleHttpFileQuery.java b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/TestExampleHttpFileQuery.java new file mode 100644 index 0000000000..89353f8aa0 --- /dev/null +++ b/tajo-tablespace-example/src/test/java/org/apache/tajo/storage/http/TestExampleHttpFileQuery.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.storage.http; + +import net.minidev.json.JSONObject; +import org.apache.tajo.QueryTestCaseBase; +import org.apache.tajo.exception.TajoException; +import org.apache.tajo.storage.TablespaceManager; +import org.junit.*; + +import java.net.InetAddress; +import java.net.URI; + +public class TestExampleHttpFileQuery extends QueryTestCaseBase { + + private static ExampleHttpTablespaceTestServer server; + + @BeforeClass + public static void setup() throws Exception { + server = new ExampleHttpTablespaceTestServer(); + server.init(); + + JSONObject configElements = new JSONObject(); + URI uri = URI.create("http://" + InetAddress.getLocalHost().getHostName() + ":" + server.getAddress().getPort()); + TablespaceManager.addTableSpaceForTest(new ExampleHttpFileTablespace("http_example", uri, configElements)); + + QueryTestCaseBase.testingCluster.getMaster().refresh(); + } + + @AfterClass + public static void teardown() throws Exception { + server.close(); + } + + @Before + public void prepareTables() throws TajoException { + executeString("create table got (*) tablespace http_example using ex_http_json with ('path'='got.json')"); + executeString("create table github (*) tablespace http_example using ex_http_json with ('path'='github.json')"); + } + + @After + public void dropTables() throws TajoException { + executeString("drop table got"); + executeString("drop table github"); + } + + @SimpleTest + @Test + public void testSelect() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + public void testGroupby() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + public void testSort() throws Exception { + runSimpleTests(); + } + + @SimpleTest + @Test + public void testJoin() throws Exception { + runSimpleTests(); + } +} diff --git a/tajo-tablespace-example/src/test/resources/dataset/github.json b/tajo-tablespace-example/src/test/resources/dataset/github.json new file mode 100644 index 0000000000..0b597b2ab4 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/dataset/github.json @@ -0,0 +1,4 @@ +{"id":"2937257753","type":"PushEvent","actor":{"id":5266949,"login":"hardrubic","gravatar_id":"","url":"https://api.github.com/users/hardrubic","avatar_url":"https://avatars.githubusercontent.com/u/5266949?"},"repo":{"id":38299397,"name":"hardrubic/rxJavaTest","url":"https://api.github.com/repos/hardrubic/rxJavaTest"},"payload":{"push_id":712081726,"size":1,"distinct_size":1,"ref":"refs/heads/master","head":"ea79d7a424f2693b70b9496726f315a5711b6fe7","before":"613f05557ad353f4bedc6df54128f8091ed1f1e9","commits":[{"sha":"ea79d7a424f2693b70b9496726f315a5711b6fe7","author":{"email":"dgzx106@163.com","name":"hardrubic"},"message":"增加rxJava例子","distinct":true,"url":"https://api.github.com/repos/hardrubic/rxJavaTest/commits/ea79d7a424f2693b70b9496726f315a5711b6fe7"}]},"public":true,"created_at":"2015-07-01T00:00:01Z"} +{"id":"2937257758","type":"WatchEvent","actor":{"id":11455393,"login":"chrischjh","gravatar_id":"","url":"https://api.github.com/users/chrischjh","avatar_url":"https://avatars.githubusercontent.com/u/11455393?"},"repo":{"id":18218031,"name":"dead-horse/co-and-koa-talk","url":"https://api.github.com/repos/dead-horse/co-and-koa-talk"},"payload":{"action":"started"},"public":true,"created_at":"2015-07-01T00:00:01Z"} +{"id":"2937257759","type":"CreateEvent","actor":{"id":206379,"login":"gvn","gravatar_id":"","url":"https://api.github.com/users/gvn","avatar_url":"https://avatars.githubusercontent.com/u/206379?"},"repo":{"id":24345476,"name":"gvn/webmaker-android","url":"https://api.github.com/repos/gvn/webmaker-android"},"payload":{"ref":"use-self-building","ref_type":"branch","master_branch":"master","description":"Webmaker for Firefox OS & Android","pusher_type":"user"},"public":true,"created_at":"2015-07-01T00:00:01Z"} +{"id":"2937257761","type":"ForkEvent","actor":{"id":1088854,"login":"CAOakleyII","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","avatar_url":"https://avatars.githubusercontent.com/u/1088854?"},"repo":{"id":11909954,"name":"skycocker/chromebrew","url":"https://api.github.com/repos/skycocker/chromebrew"},"payload":{"forkee":{"id":38339291,"name":"chromebrew","full_name":"CAOakleyII/chromebrew","owner":{"login":"CAOakleyII","id":1088854,"avatar_url":"https://avatars.githubusercontent.com/u/1088854?v=3","gravatar_id":"","url":"https://api.github.com/users/CAOakleyII","html_url":"https://github.com/CAOakleyII","followers_url":"https://api.github.com/users/CAOakleyII/followers","following_url":"https://api.github.com/users/CAOakleyII/following{/other_user}","gists_url":"https://api.github.com/users/CAOakleyII/gists{/gist_id}","starred_url":"https://api.github.com/users/CAOakleyII/starred{/owner}{/repo}","subscriptions_url":"https://api.github.com/users/CAOakleyII/subscriptions","organizations_url":"https://api.github.com/users/CAOakleyII/orgs","repos_url":"https://api.github.com/users/CAOakleyII/repos","events_url":"https://api.github.com/users/CAOakleyII/events{/privacy}","received_events_url":"https://api.github.com/users/CAOakleyII/received_events","type":"User","site_admin":false},"private":false,"html_url":"https://github.com/CAOakleyII/chromebrew","description":"Package manager for Chrome OS","fork":true,"url":"https://api.github.com/repos/CAOakleyII/chromebrew","forks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/forks","keys_url":"https://api.github.com/repos/CAOakleyII/chromebrew/keys{/key_id}","collaborators_url":"https://api.github.com/repos/CAOakleyII/chromebrew/collaborators{/collaborator}","teams_url":"https://api.github.com/repos/CAOakleyII/chromebrew/teams","hooks_url":"https://api.github.com/repos/CAOakleyII/chromebrew/hooks","issue_events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/events{/number}","events_url":"https://api.github.com/repos/CAOakleyII/chromebrew/events","assignees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/assignees{/user}","branches_url":"https://api.github.com/repos/CAOakleyII/chromebrew/branches{/branch}","tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/tags","blobs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/blobs{/sha}","git_tags_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/tags{/sha}","git_refs_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/refs{/sha}","trees_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/trees{/sha}","statuses_url":"https://api.github.com/repos/CAOakleyII/chromebrew/statuses/{sha}","languages_url":"https://api.github.com/repos/CAOakleyII/chromebrew/languages","stargazers_url":"https://api.github.com/repos/CAOakleyII/chromebrew/stargazers","contributors_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contributors","subscribers_url":"https://api.github.com/repos/CAOakleyII/chromebrew/subscribers","subscription_url":"https://api.github.com/repos/CAOakleyII/chromebrew/subscription","commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/commits{/sha}","git_commits_url":"https://api.github.com/repos/CAOakleyII/chromebrew/git/commits{/sha}","comments_url":"https://api.github.com/repos/CAOakleyII/chromebrew/comments{/number}","issue_comment_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues/comments{/number}","contents_url":"https://api.github.com/repos/CAOakleyII/chromebrew/contents/{+path}","compare_url":"https://api.github.com/repos/CAOakleyII/chromebrew/compare/{base}...{head}","merges_url":"https://api.github.com/repos/CAOakleyII/chromebrew/merges","archive_url":"https://api.github.com/repos/CAOakleyII/chromebrew/{archive_format}{/ref}","downloads_url":"https://api.github.com/repos/CAOakleyII/chromebrew/downloads","issues_url":"https://api.github.com/repos/CAOakleyII/chromebrew/issues{/number}","pulls_url":"https://api.github.com/repos/CAOakleyII/chromebrew/pulls{/number}","milestones_url":"https://api.github.com/repos/CAOakleyII/chromebrew/milestones{/number}","notifications_url":"https://api.github.com/repos/CAOakleyII/chromebrew/notifications{?since,all,participating}","labels_url":"https://api.github.com/repos/CAOakleyII/chromebrew/labels{/name}","releases_url":"https://api.github.com/repos/CAOakleyII/chromebrew/releases{/id}","created_at":"2015-07-01T00:00:00Z","updated_at":"2015-06-28T10:11:09Z","pushed_at":"2015-06-09T07:46:57Z","git_url":"git://github.com/CAOakleyII/chromebrew.git","ssh_url":"git@github.com:CAOakleyII/chromebrew.git","clone_url":"https://github.com/CAOakleyII/chromebrew.git","svn_url":"https://github.com/CAOakleyII/chromebrew","homepage":"http://skycocker.github.io/chromebrew/","size":846,"stargazers_count":0,"watchers_count":0,"language":null,"has_issues":false,"has_downloads":true,"has_wiki":true,"has_pages":false,"forks_count":0,"mirror_url":null,"open_issues_count":0,"forks":0,"open_issues":0,"watchers":0,"default_branch":"master","public":true}},"public":true,"created_at":"2015-07-01T00:00:01Z"} diff --git a/tajo-tablespace-example/src/test/resources/dataset/got.json b/tajo-tablespace-example/src/test/resources/dataset/got.json new file mode 100644 index 0000000000..db3ad6c7f0 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/dataset/got.json @@ -0,0 +1,3 @@ +{ "title" : "Hand of the King", "name" : { "first_name": "Eddard", "last_name": "Stark"}} +{ "title" : "Assassin", "name" : { "first_name": "Arya", "last_name": "Stark"}} +{ "title" : "Dancing Master", "name" : { "first_name": "Syrio", "last_name": "Forel"}} diff --git a/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testGroupby.sql b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testGroupby.sql new file mode 100644 index 0000000000..ea68caa8a1 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testGroupby.sql @@ -0,0 +1 @@ +select actor.id, count(*) from github group by actor.id; \ No newline at end of file diff --git a/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testJoin.sql b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testJoin.sql new file mode 100644 index 0000000000..e2c95c8240 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testJoin.sql @@ -0,0 +1 @@ +select g1.title from got g1, got g2 where g1.name.first_name = g2.name.first_name \ No newline at end of file diff --git a/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSelect.sql b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSelect.sql new file mode 100644 index 0000000000..80ce27f736 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSelect.sql @@ -0,0 +1 @@ +select title, name.first_name from got; \ No newline at end of file diff --git a/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSort.sql b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSort.sql new file mode 100644 index 0000000000..d31ea941dd --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/queries/TestExampleHttpFileQuery/testSort.sql @@ -0,0 +1 @@ +select actor.id, actor.login from github order by type; \ No newline at end of file diff --git a/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testGroupby.result b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testGroupby.result new file mode 100644 index 0000000000..67797c83d6 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testGroupby.result @@ -0,0 +1,6 @@ +actor/id,?count +------------------------------- +206379,1 +11455393,1 +1088854,1 +5266949,1 diff --git a/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testJoin.result b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testJoin.result new file mode 100644 index 0000000000..a6da9fc1be --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testJoin.result @@ -0,0 +1,5 @@ +title +------------------------------- +Hand of the King +Assassin +Dancing Master diff --git a/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSelect.result b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSelect.result new file mode 100644 index 0000000000..651a9feb56 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSelect.result @@ -0,0 +1,5 @@ +title,name/first_name +------------------------------- +Hand of the King,Eddard +Assassin,Arya +Dancing Master,Syrio diff --git a/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSort.result b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSort.result new file mode 100644 index 0000000000..e0073b9320 --- /dev/null +++ b/tajo-tablespace-example/src/test/resources/results/TestExampleHttpFileQuery/testSort.result @@ -0,0 +1,6 @@ +actor/id,actor/login +------------------------------- +206379,gvn +1088854,CAOakleyII +5266949,hardrubic +11455393,chrischjh diff --git a/tajo-yarn/pom.xml b/tajo-yarn/pom.xml index 70511a187b..cb472761d9 100644 --- a/tajo-yarn/pom.xml +++ b/tajo-yarn/pom.xml @@ -59,7 +59,6 @@ maven-assembly-plugin - 2.4.1 jar-with-dependencies