You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by cg...@apache.org on 2023/02/23 20:38:22 UTC

[drill] branch master updated: DRILL-8402: Add REGEXP_EXTRACT Function (#2762)

This is an automated email from the ASF dual-hosted git repository.

cgivre pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new 87638db72b DRILL-8402: Add REGEXP_EXTRACT Function (#2762)
87638db72b is described below

commit 87638db72bc4b13bcf77af1d4d14d4ea8db94947
Author: Charles S. Givre <cg...@apache.org>
AuthorDate: Thu Feb 23 15:38:14 2023 -0500

    DRILL-8402: Add REGEXP_EXTRACT Function (#2762)
---
 .../exec/expr/fn/impl/CharSequenceWrapper.java     |  11 ++-
 .../drill/exec/expr/fn/impl/StringFunctions.java   | 103 +++++++++++++++++++++
 .../exec/expr/fn/impl/TestRegexpFunctions.java     |  76 +++++++++++++++
 3 files changed, 187 insertions(+), 3 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSequenceWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSequenceWrapper.java
index 1799ee3c09..b480a5178c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSequenceWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSequenceWrapper.java
@@ -90,7 +90,9 @@ public class CharSequenceWrapper implements CharSequence {
    */
   @Override
   public CharSequence subSequence(int start, int end) {
-    throw new UnsupportedOperationException();
+    CharSequenceWrapper charSequenceWrapper = new CharSequenceWrapper();
+    charSequenceWrapper.setBuffer(start, end, buffer);
+    return charSequenceWrapper;
   }
 
   /**
@@ -212,7 +214,10 @@ public class CharSequenceWrapper implements CharSequence {
    */
   @Override
   public String toString() {
-    throw new UnsupportedOperationException();
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < length(); i++) {
+      sb.append(charAt(i));
+    }
+    return sb.toString();
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index 5b645f28ba..eeeceb8bfb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
@@ -293,6 +293,109 @@ public class StringFunctions{
     }
   }
 
+  /*
+   * This function returns the capturing groups from a regex.
+   */
+  @FunctionTemplate(name = "regexp_extract", scope = FunctionScope.SIMPLE,
+      outputWidthCalculatorType = OutputWidthCalculatorType.CUSTOM_FIXED_WIDTH_DEFAULT)
+  public static class RegexpExtract implements DrillSimpleFunc {
+
+    @Param VarCharHolder input;
+    @Param(constant=true) VarCharHolder pattern;
+    @Inject
+    DrillBuf buffer;
+    @Workspace
+    java.util.regex.Matcher matcher;
+    @Workspace
+    org.apache.drill.exec.expr.fn.impl.CharSequenceWrapper charSequenceWrapper;
+    @Output
+    ComplexWriter out;
+
+    @Override
+    public void setup() {
+      matcher = java.util.regex.Pattern.compile(org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(pattern.start,  pattern.end,  pattern.buffer)).matcher("");
+      charSequenceWrapper = new org.apache.drill.exec.expr.fn.impl.CharSequenceWrapper();
+      matcher.reset(charSequenceWrapper);
+    }
+
+    @Override
+    public void eval() {
+      charSequenceWrapper.setBuffer(input.start, input.end, input.buffer);
+
+      // Reusing same charSequenceWrapper, no need to pass it in.
+      matcher.reset();
+      boolean result = matcher.find();
+
+      // Start the list here.  If there are no matches, we return an empty list.
+      org.apache.drill.exec.vector.complex.writer.BaseWriter.ListWriter listWriter = out.rootAsList();
+      listWriter.startList();
+
+      if (result) {
+        org.apache.drill.exec.vector.complex.writer.VarCharWriter varCharWriter = listWriter.varChar();
+
+        for(int i = 1; i <= matcher.groupCount(); i++) {
+          final byte[] strBytes = matcher.group(i).getBytes(com.google.common.base.Charsets.UTF_8);
+          buffer = buffer.reallocIfNeeded(strBytes.length);
+          buffer.setBytes(0, strBytes);
+          varCharWriter.writeVarChar(0, strBytes.length, buffer);
+        }
+      }
+      listWriter.endList();
+    }
+  }
+
+  /*
+   * This function returns a specific capturing group from a regex.
+   */
+  @FunctionTemplate(name = "regexp_extract", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RegexpExtractWithIndex implements DrillSimpleFunc {
+
+    @Param VarCharHolder input;
+    @Param(constant=true) VarCharHolder pattern;
+    @Param(constant=true) IntHolder indexHolder;
+
+    @Inject
+    DrillBuf buffer;
+
+    @Workspace
+    java.util.regex.Matcher matcher;
+
+    @Workspace
+    org.apache.drill.exec.expr.fn.impl.CharSequenceWrapper charSequenceWrapper;
+
+    @Workspace
+    int index;
+
+    @Output
+    VarCharHolder out;
+
+    @Override
+    public void setup() {
+      matcher = java.util.regex.Pattern.compile(org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(pattern.start,  pattern.end,  pattern.buffer)).matcher("");
+      charSequenceWrapper = new org.apache.drill.exec.expr.fn.impl.CharSequenceWrapper();
+      matcher.reset(charSequenceWrapper);
+      index = indexHolder.value;
+    }
+
+    @Override
+    public void eval() {
+      charSequenceWrapper.setBuffer(input.start, input.end, input.buffer);
+
+      // Reusing same charSequenceWrapper, no need to pass it in.
+      matcher.reset();
+      boolean result = matcher.find();
+
+      if (result) {
+        byte[] strBytes = matcher.group(index).getBytes(com.google.common.base.Charsets.UTF_8);
+
+        out.buffer = buffer = buffer.reallocIfNeeded(strBytes.length);
+        out.start = 0;
+        out.end = strBytes.length;
+        out.buffer.setBytes(0, strBytes);
+      }
+    }
+  }
+
   @FunctionTemplate(names = {"char_length", "character_length", "length"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class CharLength implements DrillSimpleFunc {
     @Param  VarCharHolder input;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestRegexpFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestRegexpFunctions.java
new file mode 100644
index 0000000000..520e59d345
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestRegexpFunctions.java
@@ -0,0 +1,76 @@
+/*
+ * 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.drill.exec.expr.fn.impl;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.drill.test.rowSet.RowSetUtilities.strArray;
+
+public class TestRegexpFunctions extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+  }
+
+  @Test
+  public void testRegexpExtraction() throws Exception {
+    String sql = "SELECT regexp_extract('123-456-789', '([0-9]{3})-([0-9]{3})-([0-9]{3})') AS extractedText, " +
+        "regexp_extract('123-456-789', '[0-9]{3}-[0-9]{3}-[0-9]{3}') AS none";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("extractedText", MinorType.VARCHAR, DataMode.REPEATED)
+        .add("none", MinorType.VARCHAR, DataMode.REPEATED)
+        .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+        .addRow(strArray("123", "456", "789"), strArray())
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+
+  @Test
+  public void testRegexpExtractionWithIndex() throws Exception {
+    String sql = "SELECT regexp_extract('123-456-789', '([0-9]{3})-([0-9]{3})-([0-9]{3})', 1) AS extractedText, " +
+        "regexp_extract('123-456-789', '([0-9]{3})-([0-9]{3})-([0-9]{3})', 0) AS allText";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+        .add("extractedText", MinorType.VARCHAR)
+        .add("allText", MinorType.VARCHAR)
+        .buildSchema();
+
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+        .addRow("123", "123-456-789")
+        .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
+  }
+}