You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@orc.apache.org by Andrey Elenskiy <an...@arista.com> on 2021/01/19 19:19:48 UTC

[Java] PhysicalWriter to DataOutputStream implementation?

Hello, currently there's only a single implementation of PhysicalWriter
that I were able to find -- PhysicalFSWriter, which only gives the option
to write to HDFS.

I'd like to reuse the ORC file format for my own purposes without the
destination being HDFS, but just some byte buffer where I can decide myself
where the bytes end up being saved.

I've started implementing PhysicalWriter, but it seems like a lot of it
just ends up being copied over from PhysicalFSWriter which seems redundant.
So, I'm wondering if maybe something already exists to achieve my goal of
just writing resulting columns to DataOutputStream (maybe there's some
unofficial Java library or I'm missing some obvious official API).

Thanks,
Andrey

Re: [Java] PhysicalWriter to DataOutputStream implementation?

Posted by István <le...@gmail.com>.
Hi Andrey,

This is an older code, you can adjust it to a more up-to-date version. As
far as I know, ORC does not have anything that you are looking for where
Hadoop is separated from ORC. Maybe the C++ version has that. I was trying
to get rid of Hadoop from ORC years ago but never finished the project.
Generally speaking, non-Java projects have Hadoop/HDFS free version for
these columnar formats. I use C# with Parquet that does exactly that. Not
sure about ORC.


package com.streambright.orcdemo;

import org.apache.hadoop.hive.ql.io.orc.Writer;
import org.apache.hadoop.hive.ql.io.orc.OrcFile;
import org.apache.hadoop.hive.ql.io.orc.CompressionKind;

import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;

import org.apache.hadoop.conf.Configuration;

import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import java.io.IOException;
import java.util.UUID;


public class App {

    private static Configuration conf = new Configuration(false);
    private static Writer writer;
    private static OrcFile.WriterOptions orc_options =
OrcFile.writerOptions(conf);
    private static OrcFile.Version vers = OrcFile.Version.V_0_12;
    private static CompressionKind compr = CompressionKind.ZLIB;

    public static class OrcRow {
        Integer col1;
        String col2;
        String col3;

        OrcRow(int a, String b, String c) {
            this.col1 = a;
            this.col2 = b;
            this.col3 = c;
        }
    }

    public static void main(String[] args) throws IOException,
InterruptedException, ClassNotFoundException {

        String path = "/tmp/orcfile.orc";

        try {

            conf = new Configuration();
            FileSystem fs = FileSystem.getLocal(conf);

            ObjectInspector ObjInspector =
ObjectInspectorFactory.getReflectionObjectInspector(OrcRow.class,
ObjectInspectorFactory.ObjectInspectorOptions.JAVA);

            Path local_path = new Path(path);

            orc_options.inspector(ObjInspector).
                    stripeSize(8388608).
                    bufferSize(8388608).
                    blockPadding(true).
                    bloomFilterColumns("col1").
                    compress(compr).
                    version(vers);

            writer = OrcFile.createWriter(local_path, orc_options);
            //reader = OrcFile.createReader(fs, local_path);

            for (int i=1; i<1100000; i++) {
                writer.addRow(new OrcRow(i, UUID.randomUUID().toString(),
"orcFile"));
            }
            writer.close();
        } catch (Exception e) {
            e.printStackTrace();
        }
    }
}

Regards,
Istvan


On Tue, Jan 19, 2021 at 8:20 PM Andrey Elenskiy <an...@arista.com>
wrote:

> Hello, currently there's only a single implementation of PhysicalWriter
> that I were able to find -- PhysicalFSWriter, which only gives the option
> to write to HDFS.
>
> I'd like to reuse the ORC file format for my own purposes without the
> destination being HDFS, but just some byte buffer where I can decide myself
> where the bytes end up being saved.
>
> I've started implementing PhysicalWriter, but it seems like a lot of it
> just ends up being copied over from PhysicalFSWriter which seems redundant.
> So, I'm wondering if maybe something already exists to achieve my goal of
> just writing resulting columns to DataOutputStream (maybe there's some
> unofficial Java library or I'm missing some obvious official API).
>
> Thanks,
> Andrey
>


-- 
the sun shines for all

Re: [Java] PhysicalWriter to DataOutputStream implementation?

Posted by Owen O'Malley <ow...@gmail.com>.
Ok, I have a PR up that creates a new non-hadoop API. It also includes a
port of the tool that demonstrates reading and write ORC without hadoop on
the classpath at all.

https://github.com/apache/orc/pull/641

Check it out and let me know if it works for you.

.. Owen

On Fri, Jan 22, 2021 at 6:32 PM Andrey Elenskiy <an...@arista.com>
wrote:

> Thanks to both of you, I've actually went ahead with implementing
> FileSystemAPI following this util:
> https://github.com/apache/orc/blob/master/java/core/src/java/org/apache/orc/util/StreamWrapperFileSystem.java
> I think it would be awesome to have ORC separated from hadoop class
> eventually as I have to pull those jars as dependency and of course there
> are multiple layers of indirection here.
>
> On Fri, Jan 22, 2021 at 10:21 AM Owen O'Malley <ow...@gmail.com>
> wrote:
>
>> Ok, a couple of things:
>>
>>    - The PhysicalWriter was intended so that LLAP could implement a
>>    write through cache where the new file was put into the cache as well as
>>    written to long term storage.
>>    - The Hadoop FileSystem API, which is what ORC currently uses, is
>>    extensible and has a lot of bindings other than HDFS. For your use case,
>>    you probably want to use "file:///my-dir/my.orc"
>>    - Somewhere in the unit tests there is an implementation of Hadoop
>>    FileSystem that uses ByteBuffers in memory.
>>    - Finally, over the years there has been an ask for using ORC core
>>    without having Hadoop on the class path. Let me take a pass at that today
>>    to see if I can make that work. See
>>    https://issues.apache.org/jira/browse/ORC-508 .
>>
>> .. Owen
>>
>> On Tue, Jan 19, 2021 at 7:20 PM Andrey Elenskiy <
>> andrey.elenskiy@arista.com> wrote:
>>
>>> Hello, currently there's only a single implementation of PhysicalWriter
>>> that I were able to find -- PhysicalFSWriter, which only gives the option
>>> to write to HDFS.
>>>
>>> I'd like to reuse the ORC file format for my own purposes without the
>>> destination being HDFS, but just some byte buffer where I can decide myself
>>> where the bytes end up being saved.
>>>
>>> I've started implementing PhysicalWriter, but it seems like a lot of it
>>> just ends up being copied over from PhysicalFSWriter which seems redundant.
>>> So, I'm wondering if maybe something already exists to achieve my goal of
>>> just writing resulting columns to DataOutputStream (maybe there's some
>>> unofficial Java library or I'm missing some obvious official API).
>>>
>>> Thanks,
>>> Andrey
>>>
>>

Re: [Java] PhysicalWriter to DataOutputStream implementation?

Posted by Andrey Elenskiy <an...@arista.com>.
Thanks to both of you, I've actually went ahead with implementing
FileSystemAPI following this util:
https://github.com/apache/orc/blob/master/java/core/src/java/org/apache/orc/util/StreamWrapperFileSystem.java
I think it would be awesome to have ORC separated from hadoop class
eventually as I have to pull those jars as dependency and of course there
are multiple layers of indirection here.

On Fri, Jan 22, 2021 at 10:21 AM Owen O'Malley <ow...@gmail.com>
wrote:

> Ok, a couple of things:
>
>    - The PhysicalWriter was intended so that LLAP could implement a write
>    through cache where the new file was put into the cache as well as written
>    to long term storage.
>    - The Hadoop FileSystem API, which is what ORC currently uses, is
>    extensible and has a lot of bindings other than HDFS. For your use case,
>    you probably want to use "file:///my-dir/my.orc"
>    - Somewhere in the unit tests there is an implementation of Hadoop
>    FileSystem that uses ByteBuffers in memory.
>    - Finally, over the years there has been an ask for using ORC core
>    without having Hadoop on the class path. Let me take a pass at that today
>    to see if I can make that work. See
>    https://issues.apache.org/jira/browse/ORC-508 .
>
> .. Owen
>
> On Tue, Jan 19, 2021 at 7:20 PM Andrey Elenskiy <
> andrey.elenskiy@arista.com> wrote:
>
>> Hello, currently there's only a single implementation of PhysicalWriter
>> that I were able to find -- PhysicalFSWriter, which only gives the option
>> to write to HDFS.
>>
>> I'd like to reuse the ORC file format for my own purposes without the
>> destination being HDFS, but just some byte buffer where I can decide myself
>> where the bytes end up being saved.
>>
>> I've started implementing PhysicalWriter, but it seems like a lot of it
>> just ends up being copied over from PhysicalFSWriter which seems redundant.
>> So, I'm wondering if maybe something already exists to achieve my goal of
>> just writing resulting columns to DataOutputStream (maybe there's some
>> unofficial Java library or I'm missing some obvious official API).
>>
>> Thanks,
>> Andrey
>>
>

Re: [Java] PhysicalWriter to DataOutputStream implementation?

Posted by Owen O'Malley <ow...@gmail.com>.
Ok, a couple of things:

   - The PhysicalWriter was intended so that LLAP could implement a write
   through cache where the new file was put into the cache as well as written
   to long term storage.
   - The Hadoop FileSystem API, which is what ORC currently uses, is
   extensible and has a lot of bindings other than HDFS. For your use case,
   you probably want to use "file:///my-dir/my.orc"
   - Somewhere in the unit tests there is an implementation of Hadoop
   FileSystem that uses ByteBuffers in memory.
   - Finally, over the years there has been an ask for using ORC core
   without having Hadoop on the class path. Let me take a pass at that today
   to see if I can make that work. See
   https://issues.apache.org/jira/browse/ORC-508 .

.. Owen

On Tue, Jan 19, 2021 at 7:20 PM Andrey Elenskiy <an...@arista.com>
wrote:

> Hello, currently there's only a single implementation of PhysicalWriter
> that I were able to find -- PhysicalFSWriter, which only gives the option
> to write to HDFS.
>
> I'd like to reuse the ORC file format for my own purposes without the
> destination being HDFS, but just some byte buffer where I can decide myself
> where the bytes end up being saved.
>
> I've started implementing PhysicalWriter, but it seems like a lot of it
> just ends up being copied over from PhysicalFSWriter which seems redundant.
> So, I'm wondering if maybe something already exists to achieve my goal of
> just writing resulting columns to DataOutputStream (maybe there's some
> unofficial Java library or I'm missing some obvious official API).
>
> Thanks,
> Andrey
>