flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From twalthr <...@git.apache.org>
Subject [GitHub] flink pull request #5043: [FLINK-2170] [connectors] Add OrcRowInputFormat an...
Date Wed, 22 Nov 2017 11:12:57 GMT
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5043#discussion_r152533796
  
    --- Diff: flink-connectors/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java
---
    @@ -0,0 +1,429 @@
    +/*
    + * 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.flink.orc;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.typeutils.RowTypeInfo;
    +import org.apache.flink.orc.OrcRowInputFormat.Predicate;
    +import org.apache.flink.table.api.TableSchema;
    +import org.apache.flink.table.expressions.Attribute;
    +import org.apache.flink.table.expressions.BinaryComparison;
    +import org.apache.flink.table.expressions.EqualTo;
    +import org.apache.flink.table.expressions.Expression;
    +import org.apache.flink.table.expressions.GreaterThan;
    +import org.apache.flink.table.expressions.GreaterThanOrEqual;
    +import org.apache.flink.table.expressions.IsFalse;
    +import org.apache.flink.table.expressions.IsNotNull;
    +import org.apache.flink.table.expressions.IsNull;
    +import org.apache.flink.table.expressions.IsTrue;
    +import org.apache.flink.table.expressions.LessThan;
    +import org.apache.flink.table.expressions.LessThanOrEqual;
    +import org.apache.flink.table.expressions.Literal;
    +import org.apache.flink.table.expressions.Not;
    +import org.apache.flink.table.expressions.NotEqualTo;
    +import org.apache.flink.table.expressions.Or;
    +import org.apache.flink.table.expressions.UnaryExpression;
    +import org.apache.flink.table.sources.BatchTableSource;
    +import org.apache.flink.table.sources.FilterableTableSource;
    +import org.apache.flink.table.sources.ProjectableTableSource;
    +import org.apache.flink.table.sources.TableSource;
    +import org.apache.flink.types.Row;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
    +import org.apache.orc.TypeDescription;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.List;
    +
    +/**
    + * A TableSource to read ORC files.
    + *
    + * <p>The {@link OrcTableSource} supports projection and filter push-down.</p>
    + *
    + * <p>An {@link OrcTableSource} is used as shown in the example below.
    + *
    + * <pre>
    + * {@code
    + * String path = "file:///my/data/file.orc";
    + * String schema = "struct<col1:boolean,col2:tinyint,col3:smallint,col4:int>"
    + * OrcTableSource orcSrc = new OrcTableSource(path, schema);
    + * tEnv.registerTableSource("orcTable", orcSrc);
    + * Table res = tableEnv.sql("SELECT * FROM orcTable");
    + * }
    + * </pre>
    + */
    +public class OrcTableSource
    +	implements BatchTableSource<Row>, ProjectableTableSource<Row>, FilterableTableSource<Row>
{
    +
    +	private static final int DEFAULT_BATCH_SIZE = 1024;
    +
    +	// path to read ORC files from
    +	private final String path;
    +	// schema of the ORC file
    +	private final TypeDescription orcSchema;
    +	// the schema of the Table
    +	private final TableSchema tableSchema;
    +	// the configuration to read the file
    +	private final Configuration orcConfig;
    +	// the number of rows to read in a batch
    +	private final int batchSize;
    +
    +	// type information of the data returned by the InputFormat
    +	private final RowTypeInfo typeInfo;
    +	// list of selected ORC fields to return
    +	private final int[] selectedFields;
    +	// list of predicates to apply
    +	private final Predicate[] predicates;
    +
    +	/**
    +	 * Creates an OrcTableSouce with empty configuration.
    +	 *
    +	 * @param path      The path to read the ORC files from.
    +	 * @param orcSchema The schema of the ORC files as String.
    +	 */
    +	public OrcTableSource(String path, String orcSchema) {
    +		this(path, orcSchema, new Configuration(), DEFAULT_BATCH_SIZE);
    +	}
    +
    +	/**
    +	 * Creates an OrcTableSource from an ORC schema string.
    +	 *
    +	 * @param path      The path to read the ORC files from.
    +	 * @param orcSchema The schema of the ORC files as String.
    +	 * @param orcConfig The configuration to read the ORC files.
    +	 */
    +	public OrcTableSource(String path, String orcSchema, Configuration orcConfig) {
    +		this(path, TypeDescription.fromString(orcSchema), orcConfig, DEFAULT_BATCH_SIZE);
    +	}
    +
    +	/**
    +	 * Creates an OrcTableSource from an ORC schema string.
    +	 *
    +	 * @param path      The path to read the ORC files from.
    +	 * @param orcSchema The schema of the ORC files as String.
    +	 * @param orcConfig The configuration to read the ORC files.
    +	 * @param batchSize The number of Rows to read in a batch, default is 1000.
    +	 */
    +	public OrcTableSource(String path, String orcSchema, Configuration orcConfig, int batchSize)
{
    +		this(path, TypeDescription.fromString(orcSchema), orcConfig, batchSize);
    +	}
    +
    +	/**
    +	 * Creates an OrcTableSouce from an ORC TypeDescription.
    +	 *
    +	 * @param path		The path to read the ORC files from.
    +	 * @param orcSchema The schema of the ORC files as TypeDescription.
    +	 * @param orcConfig The configuration to read the ORC files.
    +	 * @param batchSize The number of Rows to read in a batch, default is 1000.
    +	 */
    +	public OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig,
int batchSize) {
    +		this(path, orcSchema, orcConfig, batchSize, null, null);
    +	}
    +
    +	private OrcTableSource(String path, TypeDescription orcSchema, Configuration orcConfig,
    +							int batchSize, int[] selectedFields, Predicate[] predicates) {
    +
    +		this.path = path;
    +		this.orcSchema = orcSchema;
    +		this.orcConfig = orcConfig;
    +		this.batchSize = batchSize;
    +		this.selectedFields = selectedFields;
    +		this.predicates = predicates;
    +
    +		// determine the type information from the ORC schema
    +		RowTypeInfo typeInfoFromSchema = (RowTypeInfo) OrcUtils.schemaToTypeInfo(this.orcSchema);
    +
    +		// set return type info
    +		if (selectedFields == null) {
    +			this.typeInfo = typeInfoFromSchema;
    +		} else {
    +			this.typeInfo = RowTypeInfo.projectFields(typeInfoFromSchema, selectedFields);
    +		}
    +
    +		// create a TableSchema that corresponds to the ORC schema
    +		this.tableSchema = new TableSchema(
    +			typeInfoFromSchema.getFieldNames(),
    +			typeInfoFromSchema.getFieldTypes()
    +		);
    +	}
    +
    +	@Override
    +	public DataSet<Row> getDataSet(ExecutionEnvironment execEnv) {
    +		OrcRowInputFormat orcIF = buildOrcInputFormat();
    +		if (selectedFields != null) {
    +			orcIF.selectFields(selectedFields);
    +		}
    +		if (predicates != null) {
    +			for (OrcRowInputFormat.Predicate pred : predicates) {
    +				orcIF.addPredicate(pred);
    +			}
    +		}
    +		return execEnv.createInput(orcIF);
    +	}
    +
    +	@VisibleForTesting
    +	protected OrcRowInputFormat buildOrcInputFormat() {
    +		return new OrcRowInputFormat(path, orcSchema, orcConfig, batchSize);
    +	}
    +
    +	@Override
    +	public TypeInformation<Row> getReturnType() {
    +		return typeInfo;
    +	}
    +
    +	@Override
    +	public TableSchema getTableSchema() {
    +		return this.tableSchema;
    +	}
    +
    +	@Override
    +	public TableSource<Row> projectFields(int[] selectedFields) {
    +		// create a copy of the OrcTableSouce with new selected fields
    +		return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, predicates);
    +	}
    +
    +	@Override
    +	public TableSource<Row> applyPredicate(List<Expression> predicates) {
    +		ArrayList<Predicate> orcPredicates = new ArrayList<>();
    +
    +		// we do not remove any predicates from the list because ORC does not fully apply predicates
    +		for (Expression pred : predicates) {
    +			Predicate orcPred = toOrcPredicate(pred);
    +			if (orcPred != null) {
    +				orcPredicates.add(orcPred);
    +			}
    +		}
    +
    +		return new OrcTableSource(path, orcSchema, orcConfig, batchSize, selectedFields, orcPredicates.toArray(new
Predicate[]{}));
    +	}
    +
    +	@Override
    +	public boolean isFilterPushedDown() {
    +		return this.predicates != null;
    +	}
    +
    +	@Override
    +	public String explainSource() {
    +		return "OrcFile[path=" + path + ", schema=" + orcSchema + ", filter=" + predicateString()
+ "]";
    +	}
    +
    +	private String predicateString() {
    +		if (predicates != null) {
    +			return "AND(" + Arrays.toString(predicates) + ")";
    +		} else {
    +			return "TRUE";
    +		}
    +	}
    +
    +	// Predicate conversion for filter push-down.
    +
    +	private Predicate toOrcPredicate(Expression pred) {
    +		if (pred instanceof Or) {
    +			Predicate c1 = toOrcPredicate(((Or) pred).left());
    +			Predicate c2 = toOrcPredicate(((Or) pred).right());
    +			if (c1 == null || c2 == null) {
    +				return null;
    +			} else {
    +				return new OrcRowInputFormat.Or(c1, c2);
    +			}
    +		} else if (pred instanceof Not) {
    +			Predicate c = toOrcPredicate(((Not) pred).child());
    +			if (c == null) {
    +				return null;
    +			} else {
    +				return new OrcRowInputFormat.Not(c);
    +			}
    +		} else if (pred instanceof BinaryComparison) {
    +
    +			BinaryComparison binComp = (BinaryComparison) pred;
    +
    +			if (!isValid(binComp)) {
    +				// not a valid predicate
    +				return null;
    +			}
    +			PredicateLeaf.Type litType = getLiteralType(binComp);
    +			if (litType == null) {
    +				// unsupported literal type
    +				return null;
    +			}
    +
    +			boolean literalOnRight = literalOnRight(binComp);
    +			String colName = getColumnName(binComp);
    +			Serializable literal = (Serializable) getLiteral(binComp);
    +
    +			if (pred instanceof EqualTo) {
    +				return new OrcRowInputFormat.Equals(colName, litType, literal);
    +			} else if (pred instanceof NotEqualTo) {
    +				return new OrcRowInputFormat.Not(
    +					new OrcRowInputFormat.Equals(colName, litType, literal));
    +			} else if (pred instanceof GreaterThan) {
    +				if (literalOnRight) {
    +					return new OrcRowInputFormat.Not(
    +						new OrcRowInputFormat.LessThanEquals(colName, litType, literal));
    +				} else {
    +					return new OrcRowInputFormat.LessThan(colName, litType, literal);
    +				}
    +			} else if (pred instanceof GreaterThanOrEqual) {
    +				if (literalOnRight) {
    +					return new OrcRowInputFormat.Not(
    +						new OrcRowInputFormat.LessThan(colName, litType, literal));
    +				} else {
    +					return new OrcRowInputFormat.LessThanEquals(colName, litType, literal);
    +				}
    +			} else if (pred instanceof LessThan) {
    +				if (literalOnRight) {
    +					return new OrcRowInputFormat.LessThan(colName, litType, literal);
    +				} else {
    +					return new OrcRowInputFormat.Not(
    +						new OrcRowInputFormat.LessThanEquals(colName, litType, literal));
    +				}
    +			} else if (pred instanceof LessThanOrEqual) {
    +				if (literalOnRight) {
    +					return new OrcRowInputFormat.LessThanEquals(colName, litType, literal);
    +				} else {
    +					return new OrcRowInputFormat.Not(
    +						new OrcRowInputFormat.LessThan(colName, litType, literal));
    +				}
    +			} else {
    +				// unsupported predicate
    +				return null;
    +			}
    +		} else if (pred instanceof UnaryExpression) {
    +
    +			UnaryExpression unary = (UnaryExpression) pred;
    +			if (!isValid(unary)) {
    +				// not a valid predicate
    +				return null;
    +			}
    +			PredicateLeaf.Type colType = toOrcType(((UnaryExpression) pred).child().resultType());
    +			if (colType == null) {
    +				// unsupported type
    +				return null;
    +			}
    +
    +			String colName = getColumnName(unary);
    +
    +			if (pred instanceof IsNull) {
    +				return new OrcRowInputFormat.IsNull(colName, colType);
    +			} else if (pred instanceof IsNotNull) {
    +				return new OrcRowInputFormat.Not(
    +					new OrcRowInputFormat.IsNull(colName, colType));
    +			} else if (pred instanceof IsTrue) {
    +				if (colType == PredicateLeaf.Type.BOOLEAN) {
    +					return new OrcRowInputFormat.Equals(colName, colType, Boolean.TRUE);
    +				} else {
    +					return null;
    +				}
    +			} else if (pred instanceof IsFalse) {
    --- End diff --
    
    Also add `IsNotTrue` and `IsNotFalse`?


---

Mime
View raw message