You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by twalthr <gi...@git.apache.org> on 2016/04/25 15:39:28 UTC

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

GitHub user twalthr opened a pull request:

    https://github.com/apache/flink/pull/1928

    [FLINK-3786] [core] [api-extending] Add BigDecimal and BigInteger as Basic types

    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [x] General
      - The pull request references the related JIRA issue
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message
    
    - [x] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    
    This PR adds new types as well as serializers and comparators. Most of the work went into normalized key support for both.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/twalthr/flink BigDecimalIntegerSupport

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1928.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1928
    
----
commit 760d28249eb5d59ff627cd7f23afe0c0610b6779
Author: twalthr <tw...@apache.org>
Date:   2016-04-20T13:57:39Z

    [FLINK-3786] [core] [api-extending] Add BigDecimal and BigInteger as Basic types

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61443733
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigDecComparator.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigDecComparator extends BasicTypeComparator<BigDecimal> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final long SMALLEST_MAGNITUDE = Integer.MAX_VALUE;
    +
    +	private static final long LARGEST_MAGNITUDE = ((long) Integer.MIN_VALUE) - Integer.MAX_VALUE + 1;
    +
    +	public BigDecComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigDecimal bd1 = BigDecSerializer.readBigDecimal(firstSource);
    +		BigDecimal bd2 = BigDecSerializer.readBigDecimal(secondSource);
    +		int comp = bd1.compareTo(bd2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    --- End diff --
    
    `return 5`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61446453
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    --- End diff --
    
    null check required


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/1928


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61462874
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    +				target.put(offset++, (byte) (normBitLen >>> (8 * (3 - i))));
    +			}
    +		}
    +
    +		// fill remaining bytes with most significant bits
    +		for (; len > 0; len--) {
    +			byte b = 0;
    +			for (int bytePos = 0; bytePos < 8 && bytePos < bitLen; bytePos++) {
    --- End diff --
    
    Yes, you are right.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/1928#issuecomment-215339226
  
    Two builds failed, but I think that's not the reason of my PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61450677
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    +				target.put(offset++, (byte) (normBitLen >>> (8 * (3 - i))));
    +			}
    +		}
    +
    +		// fill remaining bytes with most significant bits
    +		for (; len > 0; len--) {
    +			byte b = 0;
    +			for (int bytePos = 0; bytePos < 8 && bytePos < bitLen; bytePos++) {
    --- End diff --
    
    Can you add test data to `BigIntComparatorTest` that fails on this bug?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61481048
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    --- End diff --
    
    True


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1928#issuecomment-215471977
  
    Thanks for the PR @twalthr. I added a few comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61479446
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntSerializer.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +@Internal
    +public final class BigIntSerializer extends TypeSerializerSingleton<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public static final BigIntSerializer INSTANCE = new BigIntSerializer();
    +
    +	@Override
    +	public boolean isImmutableType() {
    +		return true;
    +	}
    +
    +	@Override
    +	public BigInteger createInstance() {
    +		return BigInteger.ZERO;
    +	}
    +
    +	@Override
    +	public BigInteger copy(BigInteger from) {
    +		return from;
    +	}
    +	
    +	@Override
    +	public BigInteger copy(BigInteger from, BigInteger reuse) {
    +		return from;
    +	}
    +
    +	@Override
    +	public int getLength() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void serialize(BigInteger record, DataOutputView target) throws IOException {
    +		writeBigInteger(record, target);
    +	}
    +
    +	@Override
    +	public BigInteger deserialize(DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +	
    +	@Override
    +	public BigInteger deserialize(BigInteger reuse, DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +
    +	@Override
    +	public void copy(DataInputView source, DataOutputView target) throws IOException {
    +		copyBigInteger(source, target);
    +	}
    +
    +	@Override
    +	public boolean canEqual(Object obj) {
    +		return obj instanceof BigIntSerializer;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	//                           Static Helpers for BigInteger Serialization
    +	// --------------------------------------------------------------------------------------------
    +
    +	public static void writeBigInteger(BigInteger record, DataOutputView target) throws IOException {
    +		// null value support
    +		if (record == null) {
    +			target.writeInt(0);
    +			return;
    +		}
    +		// fast paths for 0, 1, 10
    +		else if (record == BigInteger.ZERO) {
    --- End diff --
    
    Checks with equals would be too expensive. This should only be a reference check. Kryo does the same.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61430239
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntSerializer.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +@Internal
    +public final class BigIntSerializer extends TypeSerializerSingleton<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public static final BigIntSerializer INSTANCE = new BigIntSerializer();
    +
    +	@Override
    +	public boolean isImmutableType() {
    +		return true;
    +	}
    +
    +	@Override
    +	public BigInteger createInstance() {
    +		return BigInteger.ZERO;
    +	}
    +
    +	@Override
    +	public BigInteger copy(BigInteger from) {
    +		return from;
    +	}
    +	
    +	@Override
    +	public BigInteger copy(BigInteger from, BigInteger reuse) {
    +		return from;
    +	}
    +
    +	@Override
    +	public int getLength() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void serialize(BigInteger record, DataOutputView target) throws IOException {
    +		writeBigInteger(record, target);
    +	}
    +
    +	@Override
    +	public BigInteger deserialize(DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +	
    +	@Override
    +	public BigInteger deserialize(BigInteger reuse, DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +
    +	@Override
    +	public void copy(DataInputView source, DataOutputView target) throws IOException {
    +		copyBigInteger(source, target);
    +	}
    +
    +	@Override
    +	public boolean canEqual(Object obj) {
    +		return obj instanceof BigIntSerializer;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	//                           Static Helpers for BigInteger Serialization
    +	// --------------------------------------------------------------------------------------------
    +
    +	public static void writeBigInteger(BigInteger record, DataOutputView target) throws IOException {
    +		// null value support
    +		if (record == null) {
    +			target.writeInt(0);
    +			return;
    +		}
    +		// fast paths for 0, 1, 10
    +		else if (record == BigInteger.ZERO) {
    +			target.writeInt(1);
    +			return;
    +		}
    +		else if (record == BigInteger.ONE) {
    +			target.writeInt(2);
    +			return;
    +		}
    +		else if (record == BigInteger.TEN) {
    +			target.writeInt(3);
    +			return;
    +		}
    +		// default
    +		final byte[] bytes = record.toByteArray();
    +		// the length we write is offset by four, because null and short-paths for ZERO, ONE, and TEN
    +		target.writeInt(bytes.length + 4);
    +		target.write(bytes);
    +	}
    +
    +	public static BigInteger readBigInteger(DataInputView source) throws IOException {
    +		final int len = source.readInt();
    +		if (len < 4) {
    +			switch (len) {
    +				case 0:
    +					return null;
    +				case 1:
    +					return BigInteger.ZERO;
    +				case 2:
    +					return BigInteger.ONE;
    +				case 3:
    +					return BigInteger.TEN;
    +			}
    +		}
    +		final byte[] bytes = new byte[len - 4];
    +		source.read(bytes);
    +		return new BigInteger(bytes);
    +	}
    +
    +	public static boolean copyBigInteger(DataInputView source, DataOutputView target) throws IOException {
    +		final int len = source.readInt();
    +		target.writeInt(len);
    +		if (len >= 4) {
    --- End diff --
    
    `> 4`, otherwise `0` bytes are copied.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/1928#issuecomment-215524424
  
    @fhueske Thank you very much for reviewing this bitshifting monster ;-)
    I have fixed the unintentional bugs. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61450318
  
    --- Diff: flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigIntComparatorTest.java ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.math.BigInteger;
    +import org.apache.flink.api.common.typeutils.ComparatorTestBase;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +
    +public class BigIntComparatorTest extends ComparatorTestBase<BigInteger> {
    +
    +	@Override
    +	protected TypeComparator<BigInteger> createComparator(boolean ascending) {
    +		return new BigIntComparator(ascending);
    +	}
    +
    +	@Override
    +	protected TypeSerializer<BigInteger> createSerializer() {
    +		return new BigIntSerializer();
    +	}
    +
    +	@Override
    +	protected BigInteger[] getSortedTestData() {
    +		return new BigInteger[] {
    +			new BigInteger("-8745979691234123413478523984729447"),
    --- End diff --
    
    add `null`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61446700
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigDecComparator.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigDecComparator extends BasicTypeComparator<BigDecimal> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final long SMALLEST_MAGNITUDE = Integer.MAX_VALUE;
    +
    +	private static final long LARGEST_MAGNITUDE = ((long) Integer.MIN_VALUE) - Integer.MAX_VALUE + 1;
    +
    +	public BigDecComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigDecimal bd1 = BigDecSerializer.readBigDecimal(firstSource);
    +		BigDecimal bd2 = BigDecSerializer.readBigDecimal(secondSource);
    +		int comp = bd1.compareTo(bd2);
    --- End diff --
    
    null check


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61449715
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigDecSerializer.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +@Internal
    +public final class BigDecSerializer extends TypeSerializerSingleton<BigDecimal> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public static final BigDecSerializer INSTANCE = new BigDecSerializer();
    +
    +	@Override
    +	public boolean isImmutableType() {
    +		return true;
    +	}
    +
    +	@Override
    +	public BigDecimal createInstance() {
    +		return BigDecimal.ZERO;
    +	}
    +
    +	@Override
    +	public BigDecimal copy(BigDecimal from) {
    +		return from;
    +	}
    +	
    +	@Override
    +	public BigDecimal copy(BigDecimal from, BigDecimal reuse) {
    +		return from;
    +	}
    +
    +	@Override
    +	public int getLength() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void serialize(BigDecimal record, DataOutputView target) throws IOException {
    +		// null value support
    +		if (record == null) {
    +			BigIntSerializer.writeBigInteger(null, target);
    +			return;
    +		}
    +		// fast paths for 0, 1, 10
    +		else if (record == BigDecimal.ZERO) {
    --- End diff --
    
    check with equals


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61462468
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    +				target.put(offset++, (byte) (normBitLen >>> (8 * (3 - i))));
    +			}
    +		}
    +
    +		// fill remaining bytes with most significant bits
    +		for (; len > 0; len--) {
    +			byte b = 0;
    +			for (int bytePos = 0; bytePos < 8 && bytePos < bitLen; bytePos++) {
    --- End diff --
    
    I meant the added test data should catch the bug. Once the bug is fixed, the test case should succeed. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61461901
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    +				target.put(offset++, (byte) (normBitLen >>> (8 * (3 - i))));
    +			}
    +		}
    +
    +		// fill remaining bytes with most significant bits
    +		for (; len > 0; len--) {
    +			byte b = 0;
    +			for (int bytePos = 0; bytePos < 8 && bytePos < bitLen; bytePos++) {
    --- End diff --
    
    I think I cannot add test data that brings the test to fail. Equality is always accepted by the ComparatorTestBase. E.g. returning a constant byte is also always true.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1928#issuecomment-215527988
  
    Thanks for the update and the explanations @twalthr. 
    I think it would be good to leave comments about the "intentional bugs" (missing null check and reference equality) ;-) IntelliJ pointed those out to me and others might wonder as well.
    
    Good to merge after the comments are added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61480212
  
    --- Diff: flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import org.apache.flink.api.common.typeutils.ComparatorTestBase;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +
    +public class BigDecComparatorTest extends ComparatorTestBase<BigDecimal> {
    +
    +	@Override
    +	protected TypeComparator<BigDecimal> createComparator(boolean ascending) {
    +		return new BigDecComparator(ascending);
    +	}
    +
    +	@Override
    +	protected TypeSerializer<BigDecimal> createSerializer() {
    +		return new BigDecSerializer();
    +	}
    +
    +	@Override
    +	protected BigDecimal[] getSortedTestData() {
    +		return new BigDecimal[] {
    +			new BigDecimal("-12.5E1000"),
    --- End diff --
    
    Do we really want to support `null` comparision? This would also mean to change code in `BasicTypeComparator`. I would say serialization support yes, comparision support no.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/1928#issuecomment-216151381
  
    Merging...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61443708
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigDecComparator.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigDecComparator extends BasicTypeComparator<BigDecimal> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	private static final long SMALLEST_MAGNITUDE = Integer.MAX_VALUE;
    +
    +	private static final long LARGEST_MAGNITUDE = ((long) Integer.MIN_VALUE) - Integer.MAX_VALUE + 1;
    +
    +	public BigDecComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigDecimal bd1 = BigDecSerializer.readBigDecimal(firstSource);
    +		BigDecimal bd2 = BigDecSerializer.readBigDecimal(secondSource);
    +		int comp = bd1.compareTo(bd2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing a normalized order of magnitude of the given record.
    +	 * 2 bits determine the sign (negative, zero, positive), 33 bits determine the magnitude.
    +	 * This method adds at most 5 bytes that contain information.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigDecimal record, MemorySegment target, int offset, int len) {
    +		final long signum = record.signum();
    +
    +		// order of magnitude
    +		// smallest:
    +		// scale = Integer.MAX, precision = 1 => SMALLEST_MAGNITUDE
    +		// largest:
    +		// scale = Integer.MIN, precision = Integer.MAX => LARGEST_MAGNITUDE
    +		final long mag = ((long) record.scale()) - ((long) record.precision()) + 1;
    +
    +		// normalize value range: from 0 to (SMALLEST_MAGNITUDE + -1*LARGEST_MAGNITUDE)
    +		final long normMag = -1L * LARGEST_MAGNITUDE + mag;
    +
    +		// normalize value range dependent on sign:
    +		// 0 to (SMALLEST_MAGNITUDE + -1*LARGEST_MAGNITUDE)
    +		// OR (SMALLEST_MAGNITUDE + -1*LARGEST_MAGNITUDE) to 0
    +		// --> uses at most 33 bit (5 least-significant bytes)
    +		long signNormMag = signum < 0 ? normMag : (SMALLEST_MAGNITUDE + -1L * LARGEST_MAGNITUDE - normMag);
    +
    +		// zero has no magnitude
    +		// set 34th bit to flag zero
    +		if (signum == 0) {
    +			signNormMag = 0L;
    +			signNormMag |= (1L << 34);
    +		}
    +		// set 35th bit to flag positive sign
    +		else if (signum > 0) {
    +			signNormMag |= (1L << 35);
    +		}
    +
    +		// add 5 least-significant bytes that contain value to target
    +		for (int i = 0; i < 5 && len > 0; i++, len--) {
    +			final byte b = (byte) (signNormMag >>> (8 * (4 - i)));
    +			target.put(offset++, b);
    +		}
    +
    +		// fill remaining bytes with 0
    +		for (; len > 0; len--) {
    --- End diff --
    
    Padding can be removed if `getNormalizeKeyLen()` returns 5.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61449670
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigDecSerializer.java ---
    @@ -0,0 +1,137 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +@Internal
    +public final class BigDecSerializer extends TypeSerializerSingleton<BigDecimal> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public static final BigDecSerializer INSTANCE = new BigDecSerializer();
    +
    +	@Override
    +	public boolean isImmutableType() {
    +		return true;
    +	}
    +
    +	@Override
    +	public BigDecimal createInstance() {
    +		return BigDecimal.ZERO;
    +	}
    +
    +	@Override
    +	public BigDecimal copy(BigDecimal from) {
    +		return from;
    +	}
    +	
    +	@Override
    +	public BigDecimal copy(BigDecimal from, BigDecimal reuse) {
    +		return from;
    +	}
    +
    +	@Override
    +	public int getLength() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void serialize(BigDecimal record, DataOutputView target) throws IOException {
    +		// null value support
    +		if (record == null) {
    +			BigIntSerializer.writeBigInteger(null, target);
    +			return;
    +		}
    +		// fast paths for 0, 1, 10
    +		else if (record == BigDecimal.ZERO) {
    +			BigIntSerializer.writeBigInteger(BigInteger.ZERO, target);
    +			target.writeInt(0);
    +			return;
    +		}
    +		else if (record == BigDecimal.ONE) {
    +			BigIntSerializer.writeBigInteger(BigInteger.ONE, target);
    +			target.writeInt(0);
    +			return;
    +		}
    +		else if (record == BigDecimal.TEN) {
    +			BigIntSerializer.writeBigInteger(BigInteger.TEN, target);
    +			target.writeInt(0);
    +			return;
    +		}
    +		// default
    +		BigIntSerializer.writeBigInteger(record.unscaledValue(), target);
    +		target.writeInt(record.scale());
    +	}
    +
    +	@Override
    +	public BigDecimal deserialize(DataInputView source) throws IOException {
    +		return readBigDecimal(source);
    +	}
    +
    +	@Override
    +	public BigDecimal deserialize(BigDecimal reuse, DataInputView source) throws IOException {
    +		return readBigDecimal(source);
    +	}
    +
    +	@Override
    +	public void copy(DataInputView source, DataOutputView target) throws IOException {
    +		final boolean isNull = BigIntSerializer.copyBigInteger(source, target);
    +		if (!isNull) {
    +			final int scale = source.readInt();
    +			target.writeInt(scale);
    +		}
    +	}
    +
    +	@Override
    +	public boolean canEqual(Object obj) {
    +		return obj instanceof BigDecSerializer;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	//                           Static Helpers for BigInteger Serialization
    +	// --------------------------------------------------------------------------------------------
    +
    +	public static BigDecimal readBigDecimal(DataInputView source) throws IOException {
    +		final BigInteger unscaledValue = BigIntSerializer.readBigInteger(source);
    +		if (unscaledValue == null) {
    +			return null;
    +		}
    +		final int scale = source.readInt();
    +		// fast-path for 0, 1, 10
    +		if (scale == 0) {
    +			if (unscaledValue == BigInteger.ZERO) {
    --- End diff --
    
    Check with equals


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61449593
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntSerializer.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +@Internal
    +public final class BigIntSerializer extends TypeSerializerSingleton<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public static final BigIntSerializer INSTANCE = new BigIntSerializer();
    +
    +	@Override
    +	public boolean isImmutableType() {
    +		return true;
    +	}
    +
    +	@Override
    +	public BigInteger createInstance() {
    +		return BigInteger.ZERO;
    +	}
    +
    +	@Override
    +	public BigInteger copy(BigInteger from) {
    +		return from;
    +	}
    +	
    +	@Override
    +	public BigInteger copy(BigInteger from, BigInteger reuse) {
    +		return from;
    +	}
    +
    +	@Override
    +	public int getLength() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void serialize(BigInteger record, DataOutputView target) throws IOException {
    +		writeBigInteger(record, target);
    +	}
    +
    +	@Override
    +	public BigInteger deserialize(DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +	
    +	@Override
    +	public BigInteger deserialize(BigInteger reuse, DataInputView source) throws IOException {
    +		return readBigInteger(source);
    +	}
    +
    +	@Override
    +	public void copy(DataInputView source, DataOutputView target) throws IOException {
    +		copyBigInteger(source, target);
    +	}
    +
    +	@Override
    +	public boolean canEqual(Object obj) {
    +		return obj instanceof BigIntSerializer;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	//                           Static Helpers for BigInteger Serialization
    +	// --------------------------------------------------------------------------------------------
    +
    +	public static void writeBigInteger(BigInteger record, DataOutputView target) throws IOException {
    +		// null value support
    +		if (record == null) {
    +			target.writeInt(0);
    +			return;
    +		}
    +		// fast paths for 0, 1, 10
    +		else if (record == BigInteger.ZERO) {
    --- End diff --
    
    Checks should be done with equals. 
    ```
    BigInteger zero1 = BigInteger.ZERO;
    BigInteger zero2 = new BigInteger(0, new byte[0]);
    if (one1 != one2) { 
      // This is true!
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61476314
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    --- End diff --
    
    No, because what if length is 2. Then we can just take the first 2 bytes of integer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61435703
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    +				target.put(offset++, (byte) (normBitLen >>> (8 * (3 - i))));
    +			}
    +		}
    +
    +		// fill remaining bytes with most significant bits
    +		for (; len > 0; len--) {
    +			byte b = 0;
    +			for (int bytePos = 0; bytePos < 8 && bytePos < bitLen; bytePos++) {
    --- End diff --
    
    I think, this for loop repeatedly writes the first byte because `bytePos` is always initialized with 0.
    We need to remember how many bytes we have written.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61436480
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    +		return ascendingComparison ? comp : -comp;
    +	}
    +
    +	@Override
    +	public boolean supportsNormalizedKey() {
    +		return true;
    +	}
    +
    +	@Override
    +	public boolean supportsSerializationWithKeyNormalization() {
    +		return false;
    +	}
    +
    +	@Override
    +	public int getNormalizeKeyLen() {
    +		return Integer.MAX_VALUE;
    +	}
    +
    +	@Override
    +	public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
    +		return true;
    +	}
    +
    +	/**
    +	 * Adds a normalized key containing the normalized number of bits and MSBs of the given record.
    +	 * 1 bit determines the sign (negative, zero/positive), 31 bit the bit length of the record.
    +	 * Remaining bytes contain the most significant bits of the record.
    +	 */
    +	@Override
    +	public void putNormalizedKey(BigInteger record, MemorySegment target, int offset, int len) {
    +		// add normalized bit length (the larger the length, the larger the value)
    +		int bitLen = 0;
    +		if (len > 0) {
    +			final int signum = record.signum();
    +			bitLen = record.bitLength();
    +
    +			// normalize dependent on sign
    +			// from 0 to Integer.MAX
    +			// OR from Integer.MAX to 0
    +			int normBitLen = signum < 0 ? Integer.MAX_VALUE - bitLen : bitLen;
    +
    +			// add sign
    +			if (signum >= 0) {
    +				normBitLen |= (1 << 31);
    +			}
    +
    +			for (int i = 0; i < 4 && len > 0; i++, len--) {
    --- End diff --
    
    We can use `target.putIntBigEndian()` for this, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61450218
  
    --- Diff: flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java ---
    @@ -0,0 +1,63 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import org.apache.flink.api.common.typeutils.ComparatorTestBase;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +
    +public class BigDecComparatorTest extends ComparatorTestBase<BigDecimal> {
    +
    +	@Override
    +	protected TypeComparator<BigDecimal> createComparator(boolean ascending) {
    +		return new BigDecComparator(ascending);
    +	}
    +
    +	@Override
    +	protected TypeSerializer<BigDecimal> createSerializer() {
    +		return new BigDecSerializer();
    +	}
    +
    +	@Override
    +	protected BigDecimal[] getSortedTestData() {
    +		return new BigDecimal[] {
    +			new BigDecimal("-12.5E1000"),
    --- End diff --
    
    add `null`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3786] [core] [api-extending] Add BigDec...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1928#discussion_r61478906
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BigIntComparator.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * 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.flink.api.common.typeutils.base;
    +
    +import java.io.IOException;
    +import java.math.BigInteger;
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.MemorySegment;
    +
    +@Internal
    +public final class BigIntComparator extends BasicTypeComparator<BigInteger> {
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	public BigIntComparator(boolean ascending) {
    +		super(ascending);
    +	}
    +
    +	@Override
    +	public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
    +		BigInteger bi1 = BigIntSerializer.readBigInteger(firstSource);
    +		BigInteger bi2 = BigIntSerializer.readBigInteger(secondSource);
    +		int comp = bi1.compareTo(bi2);
    --- End diff --
    
    This is intentional. The Comparator does not support `null` comparisions. It is the same behavior than `StringComparator` has: serialize null, but do not compare it or even check it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---