PageFilePageSourceFactory.java

/*
 * Licensed 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 com.facebook.presto.hive.pagefile;

import com.facebook.presto.common.block.BlockEncodingSerde;
import com.facebook.presto.common.predicate.TupleDomain;
import com.facebook.presto.hive.EncryptionInformation;
import com.facebook.presto.hive.HdfsEnvironment;
import com.facebook.presto.hive.HiveBatchPageSourceFactory;
import com.facebook.presto.hive.HiveColumnHandle;
import com.facebook.presto.hive.HiveFileContext;
import com.facebook.presto.hive.HiveFileSplit;
import com.facebook.presto.hive.metastore.Storage;
import com.facebook.presto.spi.ConnectorPageSource;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.SchemaTableName;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import org.joda.time.DateTimeZone;

import javax.inject.Inject;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;

import static com.facebook.presto.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT;
import static com.google.common.base.Strings.nullToEmpty;
import static com.google.common.base.Throwables.propagateIfPossible;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class PageFilePageSourceFactory
        implements HiveBatchPageSourceFactory
{
    private final HdfsEnvironment hdfsEnvironment;
    private final BlockEncodingSerde blockEncodingSerde;

    @Inject
    public PageFilePageSourceFactory(
            HdfsEnvironment hdfsEnvironment,
            BlockEncodingSerde blockEncodingSerde)
    {
        this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null");
        this.blockEncodingSerde = requireNonNull(blockEncodingSerde, "blockEncodingSerde is null");
    }

    @Override
    public Optional<? extends ConnectorPageSource> createPageSource(
            Configuration configuration,
            ConnectorSession session,
            HiveFileSplit fileSplit,
            Storage storage,
            SchemaTableName tableName,
            Map<String, String> tableParameters,
            List<HiveColumnHandle> columns,
            TupleDomain<HiveColumnHandle> effectivePredicate,
            DateTimeZone hiveStorageTimeZone,
            HiveFileContext hiveFileContext,
            Optional<EncryptionInformation> encryptionInformation,
            Optional<byte[]> rowIDPartitionComponent)
    {
        if (!PageInputFormat.class.getSimpleName().equals(storage.getStorageFormat().getInputFormat())) {
            return Optional.empty();
        }

        FSDataInputStream inputStream;
        Path path = new Path(fileSplit.getPath());
        try {
            inputStream = hdfsEnvironment.getFileSystem(session.getUser(), path, configuration).openFile(path, hiveFileContext);
        }
        catch (Exception e) {
            if (nullToEmpty(e.getMessage()).trim().equals("Filesystem closed") ||
                    e instanceof FileNotFoundException) {
                throw new PrestoException(HIVE_CANNOT_OPEN_SPLIT, e);
            }
            throw new PrestoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, fileSplit), e);
        }

        try {
            PageFilePageSource pageFilePageSource = new PageFilePageSource(inputStream, fileSplit.getStart(), fileSplit.getLength(), fileSplit.getFileSize(), blockEncodingSerde, columns);
            return Optional.of(pageFilePageSource);
        }
        catch (Throwable e) {
            try {
                inputStream.close();
            }
            catch (IOException ignored) {
            }
            propagateIfPossible(e, PrestoException.class);
            throw new PrestoException(HIVE_CANNOT_OPEN_SPLIT, splitError(e, fileSplit), e);
        }
    }

    private static String splitError(Throwable t, HiveFileSplit fileSplit)
    {
        return format("Error opening Hive split %s (offset=%s, length=%s): %s",
                fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength(), t.getMessage());
    }
}