Return-Path: X-Original-To: apmail-parquet-commits-archive@minotaur.apache.org Delivered-To: apmail-parquet-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 060BB176A4 for ; Mon, 27 Apr 2015 23:11:59 +0000 (UTC) Received: (qmail 47684 invoked by uid 500); 27 Apr 2015 23:11:59 -0000 Delivered-To: apmail-parquet-commits-archive@parquet.apache.org Received: (qmail 47627 invoked by uid 500); 27 Apr 2015 23:11:58 -0000 Mailing-List: contact commits-help@parquet.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@parquet.incubator.apache.org Delivered-To: mailing list commits@parquet.incubator.apache.org Received: (qmail 47284 invoked by uid 99); 27 Apr 2015 23:11:58 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 27 Apr 2015 23:11:58 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7DFB4E17E9; Mon, 27 Apr 2015 23:11:58 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: blue@apache.org To: commits@parquet.apache.org Date: Mon, 27 Apr 2015 23:12:09 -0000 Message-Id: <032950096cec415d8d58975e726d5c4b@git.apache.org> In-Reply-To: <190ba0de36204468a8a5a0e9d143ae5d@git.apache.org> References: <190ba0de36204468a8a5a0e9d143ae5d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [12/51] [partial] parquet-mr git commit: PARQUET-23: Rename to org.apache.parquet. http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java ---------------------------------------------------------------------- diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java new file mode 100644 index 0000000..6253c99 --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java @@ -0,0 +1,272 @@ +/* + * 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.parquet.hadoop; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.ParquetProperties.WriterVersion; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.MessageType; + +/** + * Write records to a Parquet file. + */ +public class ParquetWriter implements Closeable { + + public static final int DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024; + public static final int DEFAULT_PAGE_SIZE = 1 * 1024 * 1024; + public static final CompressionCodecName DEFAULT_COMPRESSION_CODEC_NAME = + CompressionCodecName.UNCOMPRESSED; + public static final boolean DEFAULT_IS_DICTIONARY_ENABLED = true; + public static final boolean DEFAULT_IS_VALIDATING_ENABLED = false; + public static final WriterVersion DEFAULT_WRITER_VERSION = + WriterVersion.PARQUET_1_0; + + private final InternalParquetRecordWriter writer; + + /** + * Create a new ParquetWriter. + * (with dictionary encoding enabled and validation off) + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold + * @throws IOException + * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, boolean, boolean) + */ + public ParquetWriter(Path file, WriteSupport writeSupport, CompressionCodecName compressionCodecName, int blockSize, int pageSize) throws IOException { + this(file, writeSupport, compressionCodecName, blockSize, pageSize, + DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED); + } + + /** + * Create a new ParquetWriter. + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold (both data and dictionary) + * @param enableDictionary to turn dictionary encoding on + * @param validating to turn on validation using the schema + * @throws IOException + * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean) + */ + public ParquetWriter( + Path file, + WriteSupport writeSupport, + CompressionCodecName compressionCodecName, + int blockSize, + int pageSize, + boolean enableDictionary, + boolean validating) throws IOException { + this(file, writeSupport, compressionCodecName, blockSize, pageSize, pageSize, enableDictionary, validating); + } + + /** + * Create a new ParquetWriter. + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold + * @param dictionaryPageSize the page size threshold for the dictionary pages + * @param enableDictionary to turn dictionary encoding on + * @param validating to turn on validation using the schema + * @throws IOException + * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, WriterVersion) + */ + public ParquetWriter( + Path file, + WriteSupport writeSupport, + CompressionCodecName compressionCodecName, + int blockSize, + int pageSize, + int dictionaryPageSize, + boolean enableDictionary, + boolean validating) throws IOException { + this(file, writeSupport, compressionCodecName, blockSize, pageSize, + dictionaryPageSize, enableDictionary, validating, + DEFAULT_WRITER_VERSION); + } + + /** + * Create a new ParquetWriter. + * + * Directly instantiates a Hadoop {@link org.apache.hadoop.conf.Configuration} which reads + * configuration from the classpath. + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold + * @param dictionaryPageSize the page size threshold for the dictionary pages + * @param enableDictionary to turn dictionary encoding on + * @param validating to turn on validation using the schema + * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion} + * @throws IOException + * @see #ParquetWriter(Path, WriteSupport, CompressionCodecName, int, int, int, boolean, boolean, WriterVersion, Configuration) + */ + public ParquetWriter( + Path file, + WriteSupport writeSupport, + CompressionCodecName compressionCodecName, + int blockSize, + int pageSize, + int dictionaryPageSize, + boolean enableDictionary, + boolean validating, + WriterVersion writerVersion) throws IOException { + this(file, writeSupport, compressionCodecName, blockSize, pageSize, dictionaryPageSize, enableDictionary, validating, writerVersion, new Configuration()); + } + + /** + * Create a new ParquetWriter. + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold + * @param dictionaryPageSize the page size threshold for the dictionary pages + * @param enableDictionary to turn dictionary encoding on + * @param validating to turn on validation using the schema + * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion} + * @param conf Hadoop configuration to use while accessing the filesystem + * @throws IOException + */ + public ParquetWriter( + Path file, + WriteSupport writeSupport, + CompressionCodecName compressionCodecName, + int blockSize, + int pageSize, + int dictionaryPageSize, + boolean enableDictionary, + boolean validating, + WriterVersion writerVersion, + Configuration conf) throws IOException { + this(file, ParquetFileWriter.Mode.CREATE, writeSupport, + compressionCodecName, blockSize, pageSize, dictionaryPageSize, + enableDictionary, validating, writerVersion, conf); + } + /** + * Create a new ParquetWriter. + * + * @param file the file to create + * @param mode file creation mode + * @param writeSupport the implementation to write a record to a RecordConsumer + * @param compressionCodecName the compression codec to use + * @param blockSize the block size threshold + * @param pageSize the page size threshold + * @param dictionaryPageSize the page size threshold for the dictionary pages + * @param enableDictionary to turn dictionary encoding on + * @param validating to turn on validation using the schema + * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion} + * @param conf Hadoop configuration to use while accessing the filesystem + * @throws IOException + */ + public ParquetWriter( + Path file, + ParquetFileWriter.Mode mode, + WriteSupport writeSupport, + CompressionCodecName compressionCodecName, + int blockSize, + int pageSize, + int dictionaryPageSize, + boolean enableDictionary, + boolean validating, + WriterVersion writerVersion, + Configuration conf) throws IOException { + + WriteSupport.WriteContext writeContext = writeSupport.init(conf); + MessageType schema = writeContext.getSchema(); + + ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file, + mode); + fileWriter.start(); + + CodecFactory codecFactory = new CodecFactory(conf); + CodecFactory.BytesCompressor compressor = codecFactory.getCompressor(compressionCodecName, 0); + this.writer = new InternalParquetRecordWriter( + fileWriter, + writeSupport, + schema, + writeContext.getExtraMetaData(), + blockSize, + pageSize, + compressor, + dictionaryPageSize, + enableDictionary, + validating, + writerVersion); + } + + /** + * Create a new ParquetWriter. The default block size is 50 MB.The default + * page size is 1 MB. Default compression is no compression. Dictionary encoding is disabled. + * + * @param file the file to create + * @param writeSupport the implementation to write a record to a RecordConsumer + * @throws IOException + */ + public ParquetWriter(Path file, WriteSupport writeSupport) throws IOException { + this(file, writeSupport, DEFAULT_COMPRESSION_CODEC_NAME, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE); + } + + public ParquetWriter(Path file, Configuration conf, WriteSupport writeSupport) throws IOException { + this(file, + writeSupport, + DEFAULT_COMPRESSION_CODEC_NAME, + DEFAULT_BLOCK_SIZE, + DEFAULT_PAGE_SIZE, + DEFAULT_PAGE_SIZE, + DEFAULT_IS_DICTIONARY_ENABLED, + DEFAULT_IS_VALIDATING_ENABLED, + DEFAULT_WRITER_VERSION, + conf); + } + + public void write(T object) throws IOException { + try { + writer.write(object); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void close() throws IOException { + try { + writer.close(); + } catch (InterruptedException e) { + throw new IOException(e); + } + } +} http://git-wip-us.apache.org/repos/asf/parquet-mr/blob/b10870e4/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/PrintFooter.java ---------------------------------------------------------------------- diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/PrintFooter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/PrintFooter.java new file mode 100644 index 0000000..5a3c6f5 --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/PrintFooter.java @@ -0,0 +1,271 @@ +/* + * 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.parquet.hadoop; + +import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.apache.parquet.hadoop.ParquetFileWriter.PARQUET_METADATA_FILE; + +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Deque; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HiddenFileFilter; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.MessageType; + +/** + * Utility to print footer information + * @author Julien Le Dem + * + */ +public class PrintFooter { + + public static void main(String[] args) throws Exception { + if (args.length != 1) { + System.err.println("usage PrintFooter "); + return; + } + Path path = new Path(new URI(args[0])); + final Configuration configuration = new Configuration(); + + final FileSystem fs = path.getFileSystem(configuration); + FileStatus fileStatus = fs.getFileStatus(path); + Path summary = new Path(fileStatus.getPath(), PARQUET_METADATA_FILE); + if (fileStatus.isDir() && fs.exists(summary)) { + System.out.println("reading summary file"); + FileStatus summaryStatus = fs.getFileStatus(summary); + List