You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/09/29 22:50:16 UTC

[GitHub] [pinot] SabrinaZhaozyf opened a new pull request, #9501: [WIP] Add Support for IP Address Function

SabrinaZhaozyf opened a new pull request, #9501:
URL: https://github.com/apache/pinot/pull/9501

   Label = `feature`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r993938860


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix
+   * ipPrefix is in cidr format (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress)
+      throws UnknownHostException {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    byte[] address;
+    byte[] argAddress;
+    int subnetSize;
+
+    String[] prefixLengthPair = ipPrefix.split("/");
+    try {
+      address = InetAddresses.forString(prefixLengthPair[0]).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    subnetSize = Integer.parseInt(prefixLengthPair[1]);
+    if (subnetSize < 0 || address.length * 8 < subnetSize) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    try {
+      argAddress = InetAddresses.forString(ipAddress).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress);
+    }
+    if (address.length != 4 && address.length != 16) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress);
+    }
+    if (argAddress.length != address.length) {
+      throw new IllegalArgumentException("IP type of " + ipAddress + " is different from " + ipPrefix);

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1003625003


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import inet.ipaddr.AddressStringException;
+import inet.ipaddr.IPAddress;
+import inet.ipaddr.IPAddressString;
+import inet.ipaddr.PrefixLenException;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+/**
+ * Inbuilt IP related transform functions
+ *
+ * Functions added:
+ * isSubnetOf(String ipPrefix, String ipAddress) --> boolean
+ *
+ * Functions to add:
+ * ipPrefix(String ipAddress, int prefixBits) -> String ipPrefix
+ * ipSubnetMin(String ipPrefix) -> String ipMin
+ * ipSubnetMax(String ipPrefix) -> String ipMax
+ */
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Validates IP prefix prefixStr and returns IPAddress if validated
+   */
+  private static IPAddress getPrefix(String prefixStr) {
+    IPAddressString prefix = new IPAddressString(prefixStr);
+    IPAddress prefixAddr;
+    try {
+      prefixAddr = prefix.toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + prefix);
+    }
+    if (!prefixAddr.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + prefixStr + " should be prefixed.");
+    }
+    try {
+      return prefixAddr.toPrefixBlock();
+    } catch (PrefixLenException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Validates IP address ipString and returns IPAddress if validated
+   */
+  private static IPAddress getAddress(String ipString) {
+    try {
+      return new IPAddressString(ipString).toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + ipString);
+    }
+  }
+
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress) {
+    IPAddress prefix = getPrefix(ipPrefix);
+    IPAddress ip = getAddress(ipAddress);
+    return prefix.contains(ip);
+  }
+
+  // -------------- IN HOUSE IMPLEMENTATION STARTS HERE --------------
+  private static String[] fromPrefixToPair(String ipPrefix) {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    return ipPrefix.split("/");
+  }
+
+  private static byte[] fromStringToBytes(String ipAddress) {
+    byte[] address;
+    try {
+      address = InetAddresses.forString(ipAddress).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress + " due to " + e.getMessage());
+    }
+    if (address.length != 4 && address.length != 16) {
+      throw new IllegalArgumentException(
+          "Valid IP address should have length 32 (IPv4) or 128 (IPv6). Invalid IP " + ipAddress + " has length "
+              + address.length);
+    }
+    return address;
+  }
+
+  private static byte[] getAddrMin(byte[] addr, int subnetSize) {
+    int numRangeBits = addr.length * 8 - subnetSize;
+    for (int i = 0; i < addr.length; i++) {
+      if (numRangeBits > i * 8) {
+        int shift = (numRangeBits - i * 8) < 8 ? (numRangeBits - i * 8) : 8;
+        addr[addr.length - 1 - i] &= -0x1 << shift;
+      }
+    }
+    return addr;
+  }
+
+  private static byte[] getAddrMax(byte[] addr, int subnetSize) {
+    int numRangeBits = addr.length * 8 - subnetSize;
+    for (int i = 0; i < addr.length; i++) {
+      if (numRangeBits > i * 8) {
+        int shift = (numRangeBits - i * 8) < 8 ? (numRangeBits - i * 8) : 8;
+        addr[addr.length - 1 - i] |= ~(-0x1 << shift);
+      }
+    }
+    return addr;
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOfV1(String ipPrefix, String ipAddress)
+      throws UnknownHostException {

Review Comment:
   In this PR, there are 2 paths of implementations as [discussed here](https://github.com/apache/pinot/pull/9501#issuecomment-1282966848). 
   The v1 is the in-house implementation without calling the library.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r993939035


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix
+   * ipPrefix is in cidr format (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress)
+      throws UnknownHostException {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    byte[] address;
+    byte[] argAddress;
+    int subnetSize;
+
+    String[] prefixLengthPair = ipPrefix.split("/");
+    try {
+      address = InetAddresses.forString(prefixLengthPair[0]).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    subnetSize = Integer.parseInt(prefixLengthPair[1]);
+    if (subnetSize < 0 || address.length * 8 < subnetSize) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    try {
+      argAddress = InetAddresses.forString(ipAddress).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress);
+    }
+    if (address.length != 4 && address.length != 16) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress);

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on PR #9501:
URL: https://github.com/apache/pinot/pull/9501#issuecomment-1282966848

   > Based on @kkrugler 's suggestion to explore library, I had asked @SabrinaZhaozyf to do some investigation along the following lines.
   > 
   > * Do we think library will offer comprehensive IP processing functionality besides `subnet_contains` (which is the current focus of your PR but we want to add more more in future) ?
   > * Suggest taking a look at IP related functions supported by Kusto / Presto / Snowflake / Postgres  to get an idea of IP processing landscape
   > * Do we think our impl can be simplified by using the library and be less error prone ?
   > * Probably not a good idea to use the library if it only supports specific functionality.
   > * Some systems support IP address as a data type (NETWORK_ADDRESS IIRC ). We don't want to go there (yet) but good to check once if using the library will force us down a particular approach
   > 
   > @SabrinaZhaozyf , you may want to share any of your findings here for reference / discussion.
   > 
   > @Jackie-Jiang @walterddr @xiangfu0 - both approaches are implemented here. Can we get consensus on how we move forward ? One thing is that we will have to overload this for different types. Have you seen a similar requirement ?
   
   I have summarized my research on IP processing functions in other systems and the library in this doc: https://docs.google.com/document/d/1SS04jQCojcvCrrIGJX_aHocUEXXy8rpX0uUhwAren4k/edit?usp=sharing
   
   Please feel free to provide feedback / comments and let's get consensus on how we want to proceed from here. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1016983469


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.common.function.scalar;
+
+import inet.ipaddr.AddressStringException;
+import inet.ipaddr.IPAddress;
+import inet.ipaddr.IPAddressString;
+import inet.ipaddr.PrefixLenException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+/**
+ * Inbuilt IP related transform functions
+ *
+ * Functions added:
+ * isSubnetOf(String ipPrefix, String ipAddress) --> boolean
+ *
+ * Functions to add:
+ * ipPrefix(String ipAddress, int prefixBits) -> String ipPrefix
+ * ipSubnetMin(String ipPrefix) -> String ipMin
+ * ipSubnetMax(String ipPrefix) -> String ipMax
+ */
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Validates IP prefix prefixStr and returns IPAddress if validated
+   */
+  private static IPAddress getPrefix(String prefixStr) {
+    IPAddress prefixAddr = getAddress(prefixStr);
+    if (!prefixAddr.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + prefixStr + " should be prefixed.");
+    }
+    try {
+      return prefixAddr.toPrefixBlock();
+    } catch (PrefixLenException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Validates IP address ipString and returns IPAddress if validated
+   */
+  private static IPAddress getAddress(String ipString) {
+    try {
+      return new IPAddressString(ipString).toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + ipString);
+    }
+  }
+
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress) {
+    IPAddress prefix = getPrefix(ipPrefix);
+    IPAddress ip = getAddress(ipAddress);
+    if (ip.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + ipAddress + " should not be prefixed.");
+    }
+    return prefix.contains(ip);

Review Comment:
   actually no worries. we can follow up. 
   
   - upon checking some other systems, very little has IPADDRESS built in
   - PostgreSQL throws when IP varchar cast is a malform string 
   - MySQL has a is IP address function to check if a varchar is a IP address format. 
   
   so IMO, 
   - for now this impl is good.  
   - we can provide a safety net like MySQL later: `CASE WHEN is_ipaddress(col1) AND is_ipaddress(col2) THEN isSubnetOf(col1, col2) ELSE false END`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1016961039


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.common.function.scalar;
+
+import inet.ipaddr.AddressStringException;
+import inet.ipaddr.IPAddress;
+import inet.ipaddr.IPAddressString;
+import inet.ipaddr.PrefixLenException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+/**
+ * Inbuilt IP related transform functions
+ *
+ * Functions added:
+ * isSubnetOf(String ipPrefix, String ipAddress) --> boolean
+ *
+ * Functions to add:
+ * ipPrefix(String ipAddress, int prefixBits) -> String ipPrefix
+ * ipSubnetMin(String ipPrefix) -> String ipMin
+ * ipSubnetMax(String ipPrefix) -> String ipMax
+ */
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Validates IP prefix prefixStr and returns IPAddress if validated
+   */
+  private static IPAddress getPrefix(String prefixStr) {
+    IPAddress prefixAddr = getAddress(prefixStr);
+    if (!prefixAddr.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + prefixStr + " should be prefixed.");
+    }
+    try {
+      return prefixAddr.toPrefixBlock();
+    } catch (PrefixLenException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Validates IP address ipString and returns IPAddress if validated
+   */
+  private static IPAddress getAddress(String ipString) {
+    try {
+      return new IPAddressString(ipString).toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + ipString);
+    }
+  }
+
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress) {
+    IPAddress prefix = getPrefix(ipPrefix);
+    IPAddress ip = getAddress(ipAddress);
+    if (ip.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + ipAddress + " should not be prefixed.");
+    }
+    return prefix.contains(ip);

Review Comment:
   if we allow string input then we need to handle this (e.g. malformed string should be handled. throwing here seems a bit risky)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r997544584


##########
pom.xml:
##########
@@ -1200,6 +1200,14 @@
       </dependency>
     </dependencies>
   </dependencyManagement>
+  <dependencies>

Review Comment:
   Don't add dependency on the root level. It should be added as dependency management, and  included in the sub-module



##########
pom.xml:
##########
@@ -1200,6 +1200,14 @@
       </dependency>
     </dependencies>
   </dependencyManagement>
+  <dependencies>
+    <dependency>
+      <groupId>com.github.seancfoley</groupId>
+      <artifactId>ipaddress</artifactId>
+      <version>5.3.4</version>
+      <scope>compile</scope>

Review Comment:
   (minor) compile is the default scope, which is usually omitted.



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java:
##########
@@ -114,6 +115,8 @@ public static Expression getLiteralExpression(SqlLiteral node) {
       } else {
         literal.setDoubleValue(node.bigDecimalValue().doubleValue());
       }
+    } else if (node.getTypeName().equals(SqlTypeName.BOOLEAN)) {

Review Comment:
   BOOLEAN is already handled



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] siddharthteotia commented on pull request #9501: [WIP] Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on PR #9501:
URL: https://github.com/apache/pinot/pull/9501#issuecomment-1263010775

   Context - One of our observability users wanted subnet-contains and other similar functionality for IP addresses stored as STRINGs. To be used in SQL WHERE clause mostly


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] siddharthteotia commented on pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on PR #9501:
URL: https://github.com/apache/pinot/pull/9501#issuecomment-1272167880

   @walterddr  - please take a look as well


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
walterddr commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1002517827


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import inet.ipaddr.AddressStringException;
+import inet.ipaddr.IPAddress;
+import inet.ipaddr.IPAddressString;
+import inet.ipaddr.PrefixLenException;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+/**
+ * Inbuilt IP related transform functions
+ *
+ * Functions added:
+ * isSubnetOf(String ipPrefix, String ipAddress) --> boolean
+ *
+ * Functions to add:
+ * ipPrefix(String ipAddress, int prefixBits) -> String ipPrefix
+ * ipSubnetMin(String ipPrefix) -> String ipMin
+ * ipSubnetMax(String ipPrefix) -> String ipMax
+ */
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Validates IP prefix prefixStr and returns IPAddress if validated
+   */
+  private static IPAddress getPrefix(String prefixStr) {
+    IPAddressString prefix = new IPAddressString(prefixStr);
+    IPAddress prefixAddr;
+    try {
+      prefixAddr = prefix.toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + prefix);
+    }
+    if (!prefixAddr.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + prefixStr + " should be prefixed.");
+    }
+    try {
+      return prefixAddr.toPrefixBlock();
+    } catch (PrefixLenException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Validates IP address ipString and returns IPAddress if validated
+   */
+  private static IPAddress getAddress(String ipString) {
+    try {
+      return new IPAddressString(ipString).toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + ipString);
+    }
+  }
+
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress) {
+    IPAddress prefix = getPrefix(ipPrefix);
+    IPAddress ip = getAddress(ipAddress);
+    return prefix.contains(ip);
+  }
+
+  // -------------- IN HOUSE IMPLEMENTATION STARTS HERE --------------
+  private static String[] fromPrefixToPair(String ipPrefix) {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    return ipPrefix.split("/");
+  }
+
+  private static byte[] fromStringToBytes(String ipAddress) {
+    byte[] address;
+    try {
+      address = InetAddresses.forString(ipAddress).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress + " due to " + e.getMessage());
+    }
+    if (address.length != 4 && address.length != 16) {
+      throw new IllegalArgumentException(
+          "Valid IP address should have length 32 (IPv4) or 128 (IPv6). Invalid IP " + ipAddress + " has length "
+              + address.length);
+    }
+    return address;
+  }
+
+  private static byte[] getAddrMin(byte[] addr, int subnetSize) {
+    int numRangeBits = addr.length * 8 - subnetSize;
+    for (int i = 0; i < addr.length; i++) {
+      if (numRangeBits > i * 8) {
+        int shift = (numRangeBits - i * 8) < 8 ? (numRangeBits - i * 8) : 8;
+        addr[addr.length - 1 - i] &= -0x1 << shift;
+      }
+    }
+    return addr;
+  }
+
+  private static byte[] getAddrMax(byte[] addr, int subnetSize) {
+    int numRangeBits = addr.length * 8 - subnetSize;
+    for (int i = 0; i < addr.length; i++) {
+      if (numRangeBits > i * 8) {
+        int shift = (numRangeBits - i * 8) < 8 ? (numRangeBits - i * 8) : 8;
+        addr[addr.length - 1 - i] |= ~(-0x1 << shift);
+      }
+    }
+    return addr;
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOfV1(String ipPrefix, String ipAddress)
+      throws UnknownHostException {

Review Comment:
   why do we need a v1 ? and why is it throwing exceptions?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1004909577


##########
pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/LiteralOnlyBrokerRequestTest.java:
##########
@@ -324,6 +342,29 @@ public void testBrokerRequestHandlerWithAsFunction()
     brokerResponse = requestHandler.handleRequest(request, null, requestStats);
     Assert.assertTrue(
         brokerResponse.getProcessingExceptions().get(0).getMessage().contains("IllegalArgumentException"));
+
+    request = JsonUtils.stringToJsonNode(
+        "{\"sql\":\"SELECT is_subnet_of('2001:db8:85a3::8a2e:370:7334/62', '2001:0db8:85a3:0003:ffff:ffff:ffff:ffff')"
+            + " as booleanCol\"}");
+    requestStats = Tracing.getTracer().createRequestScope();
+    brokerResponse = requestHandler.handleRequest(request, null, requestStats);
+    resultTable = brokerResponse.getResultTable();
+    dataSchema = resultTable.getDataSchema();
+    rows = resultTable.getRows();
+    Assert.assertEquals(dataSchema.getColumnName(0), "booleanCol");
+    Assert.assertEquals(dataSchema.getColumnDataType(0), DataSchema.ColumnDataType.BOOLEAN);
+    Assert.assertEquals(rows.size(), 1);
+    Assert.assertEquals(rows.get(0).length, 1);
+    Assert.assertTrue((boolean) rows.get(0)[0]);
+    Assert.assertEquals(brokerResponse.getTotalDocs(), 0);
+
+    request = JsonUtils.stringToJsonNode(

Review Comment:
   Done



##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix
+   * ipPrefix is in cidr format (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress)
+      throws UnknownHostException {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    byte[] address;
+    byte[] argAddress;
+    int subnetSize;
+
+    String[] prefixLengthPair = ipPrefix.split("/");
+    try {
+      address = InetAddresses.forString(prefixLengthPair[0]).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1016970805


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.pinot.common.function.scalar;
+
+import inet.ipaddr.AddressStringException;
+import inet.ipaddr.IPAddress;
+import inet.ipaddr.IPAddressString;
+import inet.ipaddr.PrefixLenException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+/**
+ * Inbuilt IP related transform functions
+ *
+ * Functions added:
+ * isSubnetOf(String ipPrefix, String ipAddress) --> boolean
+ *
+ * Functions to add:
+ * ipPrefix(String ipAddress, int prefixBits) -> String ipPrefix
+ * ipSubnetMin(String ipPrefix) -> String ipMin
+ * ipSubnetMax(String ipPrefix) -> String ipMax
+ */
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Validates IP prefix prefixStr and returns IPAddress if validated
+   */
+  private static IPAddress getPrefix(String prefixStr) {
+    IPAddress prefixAddr = getAddress(prefixStr);
+    if (!prefixAddr.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + prefixStr + " should be prefixed.");
+    }
+    try {
+      return prefixAddr.toPrefixBlock();
+    } catch (PrefixLenException e) {
+      throw e;
+    }
+  }
+
+  /**
+   * Validates IP address ipString and returns IPAddress if validated
+   */
+  private static IPAddress getAddress(String ipString) {
+    try {
+      return new IPAddressString(ipString).toAddress();
+    } catch (AddressStringException e) {
+      throw new IllegalArgumentException("Invalid IP Address format for " + ipString);
+    }
+  }
+
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress) {
+    IPAddress prefix = getPrefix(ipPrefix);
+    IPAddress ip = getAddress(ipAddress);
+    if (ip.isPrefixed()) {
+      throw new IllegalArgumentException("IP Address " + ipAddress + " should not be prefixed.");
+    }
+    return prefix.contains(ip);

Review Comment:
   Hi @walterddr , thanks for the comment! Could you please elaborate a bit more?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r993938726


##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix
+   * ipPrefix is in cidr format (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress)
+      throws UnknownHostException {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    byte[] address;
+    byte[] argAddress;
+    int subnetSize;
+
+    String[] prefixLengthPair = ipPrefix.split("/");
+    try {
+      address = InetAddresses.forString(prefixLengthPair[0]).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    subnetSize = Integer.parseInt(prefixLengthPair[1]);
+    if (subnetSize < 0 || address.length * 8 < subnetSize) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    try {
+      argAddress = InetAddresses.forString(ipAddress).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP: " + ipAddress);

Review Comment:
   Done.



##########
pinot-common/src/main/java/org/apache/pinot/common/function/scalar/IpAddressFunctions.java:
##########
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.pinot.common.function.scalar;
+
+import com.google.common.net.InetAddresses;
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+
+
+public class IpAddressFunctions {
+
+  private IpAddressFunctions() {
+  }
+
+  /**
+   * Returns true if ipAddress is in the subnet of ipPrefix
+   * ipPrefix is in cidr format (IPv4 or IPv6)
+   */
+  @ScalarFunction
+  public static boolean isSubnetOf(String ipPrefix, String ipAddress)
+      throws UnknownHostException {
+    if (!ipPrefix.contains("/")) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    byte[] address;
+    byte[] argAddress;
+    int subnetSize;
+
+    String[] prefixLengthPair = ipPrefix.split("/");
+    try {
+      address = InetAddresses.forString(prefixLengthPair[0]).getAddress();
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);
+    }
+    subnetSize = Integer.parseInt(prefixLengthPair[1]);
+    if (subnetSize < 0 || address.length * 8 < subnetSize) {
+      throw new IllegalArgumentException("Invalid IP prefix: " + ipPrefix);

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] SabrinaZhaozyf commented on a diff in pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
SabrinaZhaozyf commented on code in PR #9501:
URL: https://github.com/apache/pinot/pull/9501#discussion_r1004909956


##########
pom.xml:
##########
@@ -1200,6 +1200,14 @@
       </dependency>
     </dependencies>
   </dependencyManagement>
+  <dependencies>

Review Comment:
   Done



##########
pom.xml:
##########
@@ -1200,6 +1200,14 @@
       </dependency>
     </dependencies>
   </dependencyManagement>
+  <dependencies>
+    <dependency>
+      <groupId>com.github.seancfoley</groupId>
+      <artifactId>ipaddress</artifactId>
+      <version>5.3.4</version>
+      <scope>compile</scope>

Review Comment:
   Done



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/request/RequestUtils.java:
##########
@@ -114,6 +115,8 @@ public static Expression getLiteralExpression(SqlLiteral node) {
       } else {
         literal.setDoubleValue(node.bigDecimalValue().doubleValue());
       }
+    } else if (node.getTypeName().equals(SqlTypeName.BOOLEAN)) {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] kishoreg commented on pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
kishoreg commented on PR #9501:
URL: https://github.com/apache/pinot/pull/9501#issuecomment-1287946215

   I was about to suggest the same as Rong. Given that we are going toward s postgres with joins, let's emulate the functions as well.
   
   https://www.postgresql.org/docs/current/functions-net.html


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] siddharthteotia merged pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged PR #9501:
URL: https://github.com/apache/pinot/pull/9501


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] siddharthteotia commented on pull request #9501: Add Support for IP Address Function

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on PR #9501:
URL: https://github.com/apache/pinot/pull/9501#issuecomment-1281653061

   Based on @kkrugler 's suggestion to explore library, I had asked @SabrinaZhaozyf  to do some investigation along the following lines.
   
   - Do we think library will offer comprehensive IP processing functionality besides `subnet_contains` (which is the current focus of your PR but we want to add more more in future) ?
   
   - Suggest taking a look at IP related functions supported by Kusto / Presto / Snowflake / Postgres  to get an idea of IP processing landscape
   
   - Do we think our impl can be simplified by using the library and be less error prone ?
   
   - Probably not a good idea to use the library if it only supports specific functionality.
   
   - Some systems support IP address as a data type (NETWORK_ADDRESS IIRC ). We don't want to go there (yet) but good to check once if using the library will force us down a particular approach
   
   @SabrinaZhaozyf , you may want to share any of your findings here for reference / discussion.
   
   @Jackie-Jiang  @walterddr @xiangfu0 - both approaches are implemented here. Can we get consensus on how we move forward ? One thing is that we will have to overload this for different types. Have you seen a similar requirement ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org