[ https://issues.apache.org/jira/browse/PARQUET-1822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17733113#comment-17733113 ]
ASF GitHub Bot commented on PARQUET-1822: ----------------------------------------- wgtmac commented on code in PR #1111: URL: https://github.com/apache/parquet-mr/pull/1111#discussion_r1231137142 ########## parquet-common/src/main/java/org/apache/parquet/io/LocalInputFile.java: ########## @@ -0,0 +1,99 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.file.Path; + +/** + * {@code DiskInputFile} is an implementation needed by Parquet to read Review Comment: ```suggestion * {@code LocalInputFile} is an implementation needed by Parquet to read ``` ########## parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java: ########## @@ -0,0 +1,100 @@ +/* + * 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.io; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +/** + * {@code DiskOutputFile} is an implementation needed by Parquet to write + * to local data files using {@link PositionOutputStream} instances. + */ +public class LocalOutputFile implements OutputFile { + + private final Path path; + + public LocalOutputFile(Path file) { + path = file; + } + + @Override + public PositionOutputStream create(long buffer) throws IOException { + return new PositionOutputStream() { + + private final BufferedOutputStream stream = + new BufferedOutputStream(Files.newOutputStream(path), (int) buffer); Review Comment: Does this support overwrite? ########## parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java: ########## @@ -0,0 +1,100 @@ +/* + * 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.io; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +/** + * {@code DiskOutputFile} is an implementation needed by Parquet to write Review Comment: ```suggestion * {@code LocalOutputFile} is an implementation needed by Parquet to write ``` ########## parquet-common/src/main/java/org/apache/parquet/io/LocalOutputFile.java: ########## @@ -0,0 +1,100 @@ +/* + * 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.io; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +/** + * {@code DiskOutputFile} is an implementation needed by Parquet to write + * to local data files using {@link PositionOutputStream} instances. + */ +public class LocalOutputFile implements OutputFile { + + private final Path path; + + public LocalOutputFile(Path file) { + path = file; + } + + @Override + public PositionOutputStream create(long buffer) throws IOException { + return new PositionOutputStream() { + + private final BufferedOutputStream stream = + new BufferedOutputStream(Files.newOutputStream(path), (int) buffer); Review Comment: I would expect `create` fails if file with same name exists. ########## parquet-common/src/main/java/org/apache/parquet/io/LocalInputFile.java: ########## @@ -0,0 +1,99 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.file.Path; + +/** + * {@code DiskInputFile} is an implementation needed by Parquet to read + * from local data files using {@link SeekableInputStream} instances. + */ +public class LocalInputFile implements InputFile { + + private final Path path; + + public LocalInputFile(Path file) { + path = file; + } + + @Override + public long getLength() throws IOException { + RandomAccessFile file = new RandomAccessFile(path.toFile(), "r"); + long length = file.length(); Review Comment: Should it be cached in case of repeated read? Or would `path.toFile().length()` do the same thing? > Parquet without Hadoop dependencies > ----------------------------------- > > Key: PARQUET-1822 > URL: https://issues.apache.org/jira/browse/PARQUET-1822 > Project: Parquet > Issue Type: Improvement > Components: parquet-avro > Affects Versions: 1.11.0 > Environment: Amazon Fargate (linux), Windows development box. > We are writing Parquet to be read by the Snowflake and Athena databases. > Reporter: mark juchems > Priority: Minor > Labels: documentation, newbie > > I have been trying for weeks to create a parquet file from avro and write to > S3 in Java. This has been incredibly frustrating and odd as Spark can do it > easily (I'm told). > I have assembled the correct jars through luck and diligence, but now I find > out that I have to have hadoop installed on my machine. I am currently > developing in Windows and it seems a dll and exe can fix that up but am > wondering about Linus as the code will eventually run in Fargate on AWS. > *Why do I need external dependencies and not pure java?* > The thing really is how utterly complex all this is. I would like to create > an avro file and convert it to Parquet and write it to S3, but I am trapped > in "ParquetWriter" hell! > *Why can't I get a normal OutputStream and write it wherever I want?* > I have scoured the web for examples and there are a few but we really need > some documentation on this stuff. I understand that there may be reasons for > all this but I can't find them on the web anywhere. Any help? Can't we get > the "SimpleParquet" jar that does this: > > ParquetWriter writer = > AvroParquetWriter.<GenericData.Record>builder(outputStream) > .withSchema(avroSchema) > .withConf(conf) > .withCompressionCodec(CompressionCodecName.SNAPPY) > .withWriteMode(Mode.OVERWRITE)//probably not good for prod. (overwrites > files). > .build(); > -- This message was sent by Atlassian Jira (v8.20.10#820010)