Parquet-MR contains the java implementation of the Parquet format

Overview

Parquet MR Build Status

Parquet-MR contains the java implementation of the Parquet format. Parquet is a columnar storage format for Hadoop; it provides efficient storage and encoding of data. Parquet uses the record shredding and assembly algorithm described in the Dremel paper to represent nested structures.

You can find some details about the format and intended use cases in our Hadoop Summit 2013 presentation

Building

Parquet-MR uses Maven to build and depends on the thrift compiler (protoc is now managed by maven plugin).

Install Thrift

To build and install the thrift compiler, run:

wget -nv http://archive.apache.org/dist/thrift/0.15.0/thrift-0.15.0.tar.gz
tar xzf thrift-0.15.0.tar.gz
cd thrift-0.15.0
chmod +x ./configure
./configure --disable-libs
sudo make install

If you're on OSX and use homebrew, you can instead install Thrift 0.15.0 with brew and ensure that it comes first in your PATH.

brew install thrift
export PATH="/usr/local/opt/[email protected]/bin:$PATH"

Build Parquet with Maven

Once protobuf and thrift are available in your path, you can build the project by running:

LC_ALL=C mvn clean install

Features

Parquet is a very active project, and new features are being added quickly. Here are a few features:

  • Type-specific encoding
  • Hive integration (deprecated)
  • Pig integration
  • Cascading integration
  • Crunch integration
  • Apache Arrow integration
  • Apache Scrooge integration
  • Impala integration (non-nested)
  • Java Map/Reduce API
  • Native Avro support
  • Native Thrift support
  • Native Protocol Buffers support
  • Complex structure support
  • Run-length encoding (RLE)
  • Bit Packing
  • Adaptive dictionary encoding
  • Predicate pushdown
  • Column stats
  • Delta encoding
  • Index pages

Map/Reduce integration

Input and Output formats. Note that to use an Input or Output format, you need to implement a WriteSupport or ReadSupport class, which will implement the conversion of your object to and from a Parquet schema.

We've implemented this for 2 popular data formats to provide a clean migration path as well:

Thrift

Thrift integration is provided by the parquet-thrift sub-project. If you are using Thrift through Scala, you may be using Twitter's Scrooge. If that's the case, not to worry -- we took care of the Scrooge/Apache Thrift glue for you in the parquet-scrooge sub-project.

Avro

Avro conversion is implemented via the parquet-avro sub-project.

Protobuf

Protobuf conversion is implemented via the parquet-protobuf sub-project.

Create your own objects

  • The ParquetOutputFormat can be provided a WriteSupport to write your own objects to an event based RecordConsumer.
  • the ParquetInputFormat can be provided a ReadSupport to materialize your own objects by implementing a RecordMaterializer

See the APIs:

Apache Pig integration

A Loader and a Storer are provided to read and write Parquet files with Apache Pig

Storing data into Parquet in Pig is simple:

-- options you might want to fiddle with
SET parquet.page.size 1048576 -- default. this is your min read/write unit.
SET parquet.block.size 134217728 -- default. your memory budget for buffering data
SET parquet.compression lzo -- or you can use none, gzip, snappy
STORE mydata into '/some/path' USING parquet.pig.ParquetStorer;

Reading in Pig is also simple:

mydata = LOAD '/some/path' USING parquet.pig.ParquetLoader();

If the data was stored using Pig, things will "just work". If the data was stored using another method, you will need to provide the Pig schema equivalent to the data you stored (you can also write the schema to the file footer while writing it -- but that's pretty advanced). We will provide a basic automatic schema conversion soon.

Hive integration

Hive integration is provided via the parquet-hive sub-project.

Hive integration is now deprecated within the Parquet project. It is now maintained by Apache Hive.

Build

To run the unit tests: mvn test

To build the jars: mvn package

The build runs in GitHub Actions: Build Status

Add Parquet as a dependency in Maven

The current release is version 1.12.0

  <dependencies>
    <dependency>
      <groupId>org.apache.parquetgroupId>
      <artifactId>parquet-commonartifactId>
      <version>1.12.0version>
    dependency>
    <dependency>
      <groupId>org.apache.parquetgroupId>
      <artifactId>parquet-encodingartifactId>
      <version>1.12.0version>
    dependency>
    <dependency>
      <groupId>org.apache.parquetgroupId>
      <artifactId>parquet-columnartifactId>
      <version>1.12.0version>
    dependency>
    <dependency>
      <groupId>org.apache.parquetgroupId>
      <artifactId>parquet-hadoopartifactId>
      <version>1.12.0version>
    dependency>
  dependencies>

How To Contribute

We prefer to receive contributions in the form of GitHub pull requests. Please send pull requests against the parquet-mr Git repository. If you've previously forked Parquet from its old location, you will need to add a remote or update your origin remote to https://github.com/apache/parquet-mr.git

If you are looking for some ideas on what to contribute, check out jira issues for this project labeled "pick-me-up". Comment on the issue and/or contact [email protected] with your questions and ideas.

If you’d like to report a bug but don’t have time to fix it, you can still post it to our issue tracker, or email the mailing list [email protected]

To contribute a patch:

  1. Break your work into small, single-purpose patches if possible. It’s much harder to merge in a large change with a lot of disjoint features.
  2. Create a JIRA for your patch on the Parquet Project JIRA.
  3. Submit the patch as a GitHub pull request against the master branch. For a tutorial, see the GitHub guides on forking a repo and sending a pull request. Prefix your pull request name with the JIRA name (ex: https://github.com/apache/parquet-mr/pull/240).
  4. Make sure that your code passes the unit tests. You can run the tests with mvn test in the root directory.
  5. Add new unit tests for your code.

We tend to do fairly close readings of pull requests, and you may get a lot of comments. Some common issues that are not code structure related, but still important:

  • Use 2 spaces for whitespace. Not tabs, not 4 spaces. The number of the spacing shall be 2.
  • Give your operators some room. Not a+b but a + b and not foo(int a,int b) but foo(int a, int b).
  • Generally speaking, stick to the Sun Java Code Conventions
  • Make sure tests pass!

Thank you for getting involved!

Authors and contributors

Code of Conduct

We hold ourselves and the Parquet developer community to two codes of conduct:

  1. The Apache Software Foundation Code of Conduct
  2. The Twitter OSS Code of Conduct

Discussions

License

Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0 See also:

Comments
  • PARQUET-968 Add Hive/Presto support in ProtoParquet

    PARQUET-968 Add Hive/Presto support in ProtoParquet

    This PR adds Hive (https://github.com/apache/hive) and Presto (https://github.com/prestodb/presto) support for parquet messages written with ProtoParquetWriter. Hive and other tools, such as Presto (used by AWS Athena), rely on specific LIST/MAP wrappers (as defined in the parquet spec: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md). These wrappers are currently missing from the ProtoParquet schema. AvroParquet works just fine, because it adds these wrappers when it deals with arrays and maps. This PR brings these wrappers in parquet-proto, providing the same functionality that already exists in parquet-avro.

    This is backward compatible. Messages written without the extra LIST/MAP wrappers are still being read successfully using the updated ProtoParquetReader.

    Regarding the change. Given the following protobuf schema:

    message ListOfPrimitives {
        repeated int64 my_repeated_id = 1;
    }
    

    Old parquet schema was:

    message ListOfPrimitives {
      repeated int64 my_repeated_id = 1;
    }
    

    New parquet schema is:

    message ListOfPrimitives {
      required group my_repeated_id (LIST) = 1 {
        repeated group list {
          required int64 element;
        }
      }
    }
    

    For list of messages, the changes look like this:

    Protobuf schema:

    message ListOfMessages {
        string top_field = 1;
        repeated MyInnerMessage first_array = 2;
    }
    
    message MyInnerMessage {
        int32 inner_field = 1;
    }
    

    Old parquet schema was:

    message TestProto3.ListOfMessages {
      optional binary top_field (UTF8) = 1;
      repeated group first_array = 2 {
        optional int32 inner_field = 1;
      }
    }
    

    The expected parquet schema, compatible with Hive (and similar to parquet-avro) is the following (notice the LIST wrapper):

    message TestProto3.ListOfMessages {
      optional binary top_field (UTF8) = 1;
      required group first_array (LIST) = 2 {
        repeated group list {
          optional group element {
            optional int32 inner_field = 1;
          }
        }
      }
    }
    

    Similar for maps. Protobuf schema:

    message TopMessage {
        map<int64, MyInnerMessage> myMap = 1;
    }
    
    message MyInnerMessage {
        int32 inner_field = 1;
    }
    

    Old parquet schema:

    message TestProto3.TopMessage {
      repeated group myMap = 1 {
        optional int64 key = 1;
        optional group value = 2 {
          optional int32 inner_field = 1;
        }
      }
    }
    

    New parquet schema (notice the MAP wrapper):

    message TestProto3.TopMessage {
      required group myMap (MAP) = 1 {
        repeated group key_value {
          required int64 key;
          optional group value {
            optional int32 inner_field = 1;
          }
        }
      }
    }
    

    Jira: https://issues.apache.org/jira/browse/PARQUET-968

    opened by costimuraru 67
  • PARQUET-686: Allow for Unsigned Statistics in Binary Type

    PARQUET-686: Allow for Unsigned Statistics in Binary Type

    Currently, ordering of Binary in Parquet is based on byte-by-byte comparison. This doesn't match the standard method of lexicographic sorting of Unicode strings, you can see an example of this in

    • Avro: https://github.com/apache/avro/blob/master/lang/java/avro/src/main/java/org/apache/avro/io/BinaryData.java#L184
    • Spark: https://github.com/apache/spark/blob/master/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java#L835

    This overrides comparison on FromStringBinary to implement correct sort order.

    @julienledem

    opened by a10y 43
  • PARQUET-601: Add support to configure the encoding used by ValueWriters

    PARQUET-601: Add support to configure the encoding used by ValueWriters

    Context:

    Parquet is currently structured to choose the appropriate value writer based on the type of the column as well as the Parquet version. As of now, the writer(s) (and hence encoding) for each data type is hard coded in the Parquet source code.

    This PR adds support for being able to override the encodings per type via config. That allows users to experiment with various encoding strategies manually as well as enables them to override the hardcoded defaults if they don't suit their use case.

    We can override encodings per data type (int32 / int64 / ...). Something on the lines of:

    parquet.writer.encoding-override.<type> = "encoding1[,encoding2]"
    

    As an example:

    "parquet.writer.encoding-override.int32" = "plain"
    (Chooses Plain encoding and hence the PlainValuesWriter).
    

    When a primary + fallback need to be specified, we can do the following:

    "parquet.writer.encoding-override.binary" = "rle_dictionary,delta_byte_array"
    (Chooses RLE_DICTIONARY encoding as the initial encoding and DELTA_BYTE_ARRAY encoding as the fallback and hence creates a FallbackWriter(PlainBinaryDictionaryValuesWriter, DeltaByteArrayWriter). 
    

    In such cases we can mandate that the first encoding listed must allow for Fallbacks by implementing RequiresFallback.

    PR notes:

    • Restructured the ValuesWriter creation code. Pulled it out of ParquetProperties into a new class and refactored the flow based on type as it was getting hard to follow and I felt adding the overrides would make it harder. Added a bunch of unit tests to verify the ValuesWriter we create for combinations of type, parquet version and dictionary on / off.
    • Added unit tests to verify parsing of the encoding overrides + creation of ValuesWriters based on these overrides.
    • Manually tested some encoding overrides scenarios out on Hadoop (both parquet v1, v2).
    opened by piyushnarang 37
  • PARQUET-116: Pass a filter object to user defined predicate in filter2 api

    PARQUET-116: Pass a filter object to user defined predicate in filter2 api

    Currently for creating a user defined predicate using the new filter api, no value can be passed to create a dynamic filter at runtime. This reduces the usefulness of the user defined predicate, and meaningful predicates cannot be created. We can add a generic Object value that is passed through the api, which can internally be used in the keep function of the user defined predicate for creating many different types of filters. For example, in spark sql, we can pass in a list of filter values for a where IN clause query and filter the row values based on that list.

    opened by saucam 37
  • PARQUET-1633: Fix integer overflow

    PARQUET-1633: Fix integer overflow

    This PR addresses this issue: https://issues.apache.org/jira/browse/PARQUET-1633

    I have not added unit tests, as to check overflow conditions I would need test data over 2GB in size (on disk, compressed), considerably larger in-memory and thus requiring significant CI resources.

    The issue was using an int for length field, which for parquet files with very large row_group_size (row groups over 2GB) would cause silent integer overflow, manifesting itself as negative length and an attempt to create an ArrayList with negative length.

    I have tested reading a 6.6GB parquet file with a huge row group size (works out at over 2GB) which recreated this problem, and with this modification can read the file without any issues.

    opened by eadwright 36
  • PARQUET-1441: SchemaParseException: Can't redefine: list in AvroIndexedRecordConverter

    PARQUET-1441: SchemaParseException: Can't redefine: list in AvroIndexedRecordConverter

    Parquet Avro reader can't convert such Parquet schemas, where a group field name is reused in an inner structure. The converter creates Avro record schma in this case, but in Avro record types should have a unique name, therefore the result is an invalid Avro schema. This patch fixes this case by adding a namespace for the record if the name was defined before, this way making the record names unique.

    opened by nandorKollar 32
  • PARQUET-400: Replace CompatibilityUtil with SeekableInputStream.

    PARQUET-400: Replace CompatibilityUtil with SeekableInputStream.

    This fixes PARQUET-400 by replacing CompatibilityUtil with SeekableInputStream that's implemented for hadoop-1 and hadoop-2. The benefit of this approach is that SeekableInputStream can be used for non-Hadoop file systems in the future.

    This also changes the default Hadoop version to Hadoop-2. The library is still compatible with Hadoop 1.x, but this makes building Hadoop-2 classes, like H2SeekableInputStream, much easier and removes the need for multiple hadoop versions during compilation.

    opened by rdblue 26
  • PARQUET-401: Deprecate Log and move to SLF4J Logger

    PARQUET-401: Deprecate Log and move to SLF4J Logger

    The current Log class is intended to allow swapping out logger back-ends, but SLF4J already does this. It also doesn't expose as nice of an API as SLF4J, which can handle formatting to avoid the cost of building log messages that won't be used.

    We should deprecate the org.apache.parquet.Log class and move to using SLF4J directly, instead of wrapping SLF4J (PARQUET-305).

    opened by lw-lin 26
  • PARQUET-1396: Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

    PARQUET-1396: Example of using EncryptionPropertiesFactory and DecryptionPropertiesFactory

    …ncryption

    Make sure you have checked all steps below.

    Jira

    • [ ] My PR addresses the following Parquet Jira issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
      • https://issues.apache.org/jira/browse/PARQUET-XXX
      • In case you are adding a dependency, check if the license complies with the ASF 3rd Party License Policy.

    Tests

    • [ ] My PR adds the following unit tests OR does not need testing for this extremely good reason:

    Commits

    • [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"

    Documentation

    • [ ] In case of new functionality, my PR adds documentation that describes how to use it.
      • All the public functions and the classes in the PR contain Javadoc that explain what it does
    opened by shangxinli 25
  • PARQUET-225: INT64 support for Delta Encoding

    PARQUET-225: INT64 support for Delta Encoding

    JIRA: https://issues.apache.org/jira/browse/PARQUET-225

    As of now, parquet doesn't support delta encoding for INT64. However it is planned in the format: https://github.com/Parquet/parquet-format/blob/master/Encodings.md

    The benefits of this feature are huge. For timestamps it achieves twice better compression than SNAPPY on plain encoding, and the reading is faster. This feature is actually advertised on the home page of Parquet, even though it is not yet implemented: http://parquet.incubator.apache.org/ http://image.slidesharecdn.com/hadoopsummit-140630160016-phpapp01/95/efficient-data-storage-for-analytics-with-apache-parquet-20-30-638.jpg?cb=1404162126

    This pull request adds full support for INT64. It keeps INT32 binary unchanged, even though it reads it with the INT64 implementation. That might be slower on 32 bit processors, but I don't not consider it a problem.

    Regarding testing, all test made for INT32 are ran against the new INT64 implementation and also 2 new unit tests are added specifically for INT64.

    opened by lunchev 25
  • PARQUET-480: Update for Cascading 3.0

    PARQUET-480: Update for Cascading 3.0

    The code in parquet-cascading is adapted to the API as of Cascading 2.5.3

    Some incompatible changes were introduced in Cascading 3.0. This patch forks the parquet-cascading module to also provide a parquet-cascading3 module, which is about identical save for overloads which changed from requiring a Foo to requiring a Foo<? extends JobConf>

    opened by cchepelov 23
  • add support re-encryption in ColumnEncryptor

    add support re-encryption in ColumnEncryptor

    Make sure you have checked all steps below.

    Jira

    • [ ] My PR addresses the following Parquet Jira issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
      • https://issues.apache.org/jira/browse/PARQUET-XXX
      • In case you are adding a dependency, check if the license complies with the ASF 3rd Party License Policy.

    Tests

    • [ ] My PR adds the following unit tests OR does not need testing for this extremely good reason:

    Commits

    • [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"

    Documentation

    • [ ] In case of new functionality, my PR adds documentation that describes how to use it.
      • All the public functions and the classes in the PR contain Javadoc that explain what it does
    opened by vectorijk 0
  • PARQUET-2075: Implement unified file rewriter

    PARQUET-2075: Implement unified file rewriter

    Jira

    Tests

    • Make sure all tasks pass, especially CompressionConverterTest, ColumnPrunerTest, ColumnMaskerTest, and ColumnEncryptorTest.

    Commits

    • A new ParquetRewriter is introduced to unify rewriting logic.
    • RewriteOptions is defined to provide essential settings.
    • CompressionConverter, ColumnPruner, ColumnMasker, and ColumnEncryptor have been refactored and marked as deprecated.
    opened by wgtmac 4
  • Update README.md to reflect thrift 0.17

    Update README.md to reflect thrift 0.17

    now that we have 0.17 to use

    Make sure you have checked all steps below.

    Jira

    • [ ] My PR addresses the following Parquet Jira issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
      • https://issues.apache.org/jira/browse/PARQUET-XXX
      • In case you are adding a dependency, check if the license complies with the ASF 3rd Party License Policy.

    Tests

    • [x] My PR adds the following unit tests OR does not need testing for this extremely good reason:

    Commits

    • [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"

    Documentation

    • [ ] In case of new functionality, my PR adds documentation that describes how to use it.
      • All the public functions and the classes in the PR contain Javadoc that explain what it does
    opened by Jimexist 0
  • PARQUET-2159: java17 vector parquet bit-packing decode optimization

    PARQUET-2159: java17 vector parquet bit-packing decode optimization

    The PR includes 3 aspects:

    1. Use java17 vector api to decode bit-packing , the performance gain is 4x ~ 8x according to the microbenchmark
    2. Upgrade the project to java17 to support java vector api
    3. Add ParquetReadRouter to compatible with different platform when computing engines(such as spark) read parquet.

    Jira

    • https://issues.apache.org/jira/projects/PARQUET/issues/PARQUET-2159?filter=allopenissues

    Tests

    • Add unit tests org.apache.parquet.column.values.bitpacking.TestByteBitPackingVectorLE

    Commits

    • [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"

    Documentation

    • The PR adds maven profile vector to enable generate java17 vector bit-packing decode opt. code, and run junit tests: mvn clean install -P vector
    • The PR needs Intel Ice Lake CPU to run junit tests
    • The contributors are jiangjiguang jiyu1021 guangzegu Fang-Xie , and co-contributor is jatin-bhateja
    opened by jiangjiguang 13
  • PARQUET-2213: add InputFile.newStream with a read range

    PARQUET-2213: add InputFile.newStream with a read range

    This adds a new API, InputFile.newStream, that allow specifying a range for the returned input stream. This allows the implementations (especially for cloud storages like S3) to only download the data within the specified range, instead of the whole object.

    Make sure you have checked all steps below.

    Jira

    • [ ] My PR addresses the following Parquet Jira issues and references them in the PR title. For example, "PARQUET-1234: My Parquet PR"
      • https://issues.apache.org/jira/browse/PARQUET-XXX
      • In case you are adding a dependency, check if the license complies with the ASF 3rd Party License Policy.

    Tests

    • [ ] My PR adds the following unit tests OR does not need testing for this extremely good reason:

    Commits

    • [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "How to write a good git commit message":
      1. Subject is separated from body by a blank line
      2. Subject is limited to 50 characters (not including Jira issue reference)
      3. Subject does not end with a period
      4. Subject uses the imperative mood ("add", not "adding")
      5. Body wraps at 72 characters
      6. Body explains "what" and "why", not "how"

    Documentation

    • [ ] In case of new functionality, my PR adds documentation that describes how to use it.
      • All the public functions and the classes in the PR contain Javadoc that explain what it does
    opened by sunchao 4
  • PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

    PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

    The PR adds the new ByteBuffer api and also updates ColumnChunkPageReadStore.readPage to use the new API. A few additional classes were touched (ParquetReader.Builder, BytesInput) to allow an allocator to be specified and/or to avoid ByteBuffer -> byte array copying. These changes were necessary to enable the unit test. A user option has been added to explicitly enable/disable the use of the ByteBuffer api for decryption.

    Jira

    Tests

    • Updates Unit test(s) in org.apache.parquet.crypto.TestPropertiesDrivenEncryption
    opened by parthchandra 4
Owner
The Apache Software Foundation
The Apache Software Foundation
Bloofi: A java implementation of multidimensional Bloom filters

Bloofi: A java implementation of multidimensional Bloom filters Bloom filters are probabilistic data structures commonly used for approximate membersh

Daniel Lemire 71 Nov 2, 2022
High performance Java implementation of a Cuckoo filter - Apache Licensed

Cuckoo Filter For Java This library offers a similar interface to Guava's Bloom filters. In most cases it can be used interchangeably and has addition

Mark Gunlogson 161 Dec 30, 2022
Java port of a concurrent trie hash map implementation from the Scala collections library

About This is a Java port of a concurrent trie hash map implementation from the Scala collections library. It is almost a line-by-line conversion from

null 147 Oct 31, 2022
A Java implementation of Transducers

transducers-java Transducers are composable algorithmic transformations. They are independent from the context of their input and output sources and s

null 117 Sep 29, 2022
Implementation of various string similarity and distance algorithms: Levenshtein, Jaro-winkler, n-Gram, Q-Gram, Jaccard index, Longest Common Subsequence edit distance, cosine similarity ...

java-string-similarity A library implementing different string similarity and distance measures. A dozen of algorithms (including Levenshtein edit dis

Thibault Debatty 2.5k Dec 29, 2022
Golang implementation of the Alaya protocol

Go PlatON Welcome to the PlatON-Go source code repository! This is an Ethereum-based、high-performance and high-security implementation of the PlatON p

null 23 Oct 14, 2022
High Performance data structures and utility methods for Java

Agrona Agrona provides a library of data structures and utility methods that are a common need when building high-performance applications in Java. Ma

Real Logic 2.5k Jan 5, 2023
A high performance caching library for Java

Caffeine is a high performance, near optimal caching library. For more details, see our user's guide and browse the API docs for the latest release. C

Ben Manes 13k Jan 5, 2023
Chronicle Bytes has a similar purpose to Java NIO's ByteBuffer with many extensions

Chronicle-Bytes Chronicle-Bytes Chronicle Bytes contains all the low level memory access wrappers. It is built on Chronicle Core’s direct memory and O

Chronicle Software : Open Source 334 Jan 1, 2023
An advanced, but easy to use, platform for writing functional applications in Java 8.

Getting Cyclops X (10) The latest version is cyclops:10.4.0 Stackoverflow tag cyclops-react Documentation (work in progress for Cyclops X) Integration

AOL 1.3k Dec 29, 2022
Eclipse Collections is a collections framework for Java with optimized data structures and a rich, functional and fluent API.

English | 中文 | Deutsch | Español | Ελληνικά | Français | 日本語 | Norsk (bokmål) | Português-Brasil | Русский | हिंदी Eclipse Collections is a comprehens

Eclipse Foundation 2.1k Dec 29, 2022
External-Memory Sorting in Java

Externalsortinginjava External-Memory Sorting in Java: useful to sort very large files using multiple cores and an external-memory algorithm. The vers

Daniel Lemire 235 Dec 29, 2022
A Java library for quickly and efficiently parsing and writing UUIDs

fast-uuid fast-uuid is a Java library for quickly and efficiently parsing and writing UUIDs. It yields the most dramatic performance gains when compar

Jon Chambers 142 Jan 1, 2023
Geohash utitlies in java

geo Java utility methods for geohashing. Status: production, available on Maven Central Maven site reports are here including javadoc. Add this to you

Dave Moten 386 Jan 1, 2023
Hollow is a java library and toolset for disseminating in-memory datasets from a single producer to many consumers for high performance read-only access.

Hollow Hollow is a java library and toolset for disseminating in-memory datasets from a single producer to many consumers for high performance read-on

Netflix, Inc. 1.1k Dec 25, 2022
High Performance Primitive Collections for Java

HPPC: High Performance Primitive Collections Collections of primitive types (maps, sets, stacks, lists) with open internals and an API twist (no java.

Carrot Search 890 Dec 28, 2022
Java library for the HyperLogLog algorithm

java-hll A Java implementation of HyperLogLog whose goal is to be storage-compatible with other similar offerings from Aggregate Knowledge. NOTE: This

Aggregate Knowledge (a Neustar service) 296 Dec 30, 2022
A simple integer compression library in Java

JavaFastPFOR: A simple integer compression library in Java License This code is released under the Apache License Version 2.0 http://www.apache.org/li

Daniel Lemire 487 Dec 30, 2022