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

[GitHub] tajo pull request #1046: TAJO-2179: Add a regular expression scanner and app...

GitHub user jinossy opened a pull request:

    https://github.com/apache/tajo/pull/1046

    TAJO-2179: Add a regular expression scanner and appender.

    I refer to the hive for compatibility.

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

    $ git pull https://github.com/jinossy/tajo TAJO-2179

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

    https://github.com/apache/tajo/pull/1046.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 #1046
    
----

----


---
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] tajo issue #1046: TAJO-2179: Add a regular expression scanner and appender.

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on the issue:

    https://github.com/apache/tajo/pull/1046
  
     Could anyone review this?


---
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] tajo issue #1046: TAJO-2179: Add a regular expression scanner and appender.

Posted by jinossy <gi...@git.apache.org>.
Github user jinossy commented on the issue:

    https://github.com/apache/tajo/pull/1046
  
    Thank you for your review!


---
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] tajo pull request #1046: TAJO-2179: Add a regular expression scanner and app...

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

    https://github.com/apache/tajo/pull/1046#discussion_r76532365
  
    --- Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/regex/RegexLineDeserializer.java ---
    @@ -0,0 +1,167 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.tajo.storage.regex;
    +
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.CharsetUtil;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.catalog.TableMeta;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.datum.NullDatum;
    +import org.apache.tajo.exception.InvalidTablePropertyException;
    +import org.apache.tajo.exception.TajoRuntimeException;
    +import org.apache.tajo.plan.util.PlannerUtil;
    +import org.apache.tajo.storage.FieldSerializerDeserializer;
    +import org.apache.tajo.storage.StorageConstants;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.storage.text.TextFieldSerializerDeserializer;
    +import org.apache.tajo.storage.text.TextLineDeserializer;
    +import org.apache.tajo.storage.text.TextLineParsingError;
    +import org.apache.tajo.storage.text.TextLineSerDe;
    +
    +import java.io.IOException;
    +import java.nio.charset.CharsetDecoder;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +public class RegexLineDeserializer extends TextLineDeserializer {
    +  private static final Log LOG = LogFactory.getLog(RegexLineDeserializer.class);
    +
    +  private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
    +  private FieldSerializerDeserializer fieldSerDer;
    +  private ByteBuf nullChars;
    +
    +  private int[] targetColumnIndexes;
    +  private String inputRegex;
    +  private Pattern inputPattern;
    +  // Number of rows not matching the regex
    +  private long unmatchedRows = 0;
    +  private long nextUnmatchedRows = 1;
    +  // Number of rows that match the regex but have missing groups.
    +  private long partialMatchedRows = 0;
    +  private long nextPartialMatchedRows = 1;
    +
    +  public RegexLineDeserializer(Schema schema, TableMeta meta, Column[] projected) {
    +    super(schema, meta);
    +    targetColumnIndexes = PlannerUtil.getTargetIds(schema, projected);
    +  }
    +
    +  @Override
    +  public void init() {
    +    fieldSerDer = new TextFieldSerializerDeserializer(meta);
    +    fieldSerDer.init(schema);
    +
    +    // Read the configuration parameters
    +    inputRegex = meta.getProperty(StorageConstants.TEXT_REGEX);
    +    boolean inputRegexIgnoreCase = "true".equalsIgnoreCase(
    +        meta.getProperty(StorageConstants.TEXT_REGEX_CASE_INSENSITIVE, "false"));
    +
    +    // Parse the configuration parameters
    +    if (inputRegex != null) {
    +      inputPattern = Pattern.compile(inputRegex, Pattern.DOTALL
    +          + (inputRegexIgnoreCase ? Pattern.CASE_INSENSITIVE : 0));
    +    } else {
    +      throw new TajoRuntimeException(new InvalidTablePropertyException(StorageConstants.TEXT_REGEX,
    +          "This table does not have serde property \"" + StorageConstants.TEXT_REGEX + "\"!"));
    +    }
    +
    +    if (nullChars != null) {
    +      nullChars.release();
    +    }
    +    nullChars = TextLineSerDe.getNullChars(meta);
    +  }
    +
    +
    +  @Override
    +  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError {
    +
    +    if (lineBuf == null || targetColumnIndexes.length == 0) {
    +      return;
    +    }
    +
    +    String line = decoder.decode(lineBuf.nioBuffer(lineBuf.readerIndex(), lineBuf.readableBytes())).toString();
    +    int[] projection = targetColumnIndexes;
    +
    +    // Projection
    +    int currentTarget = 0;
    +    int currentIndex = 0;
    +    Matcher m = inputPattern.matcher(line);
    +
    +    if (!m.matches()) {
    +      unmatchedRows++;
    +      if (unmatchedRows >= nextUnmatchedRows) {
    +        nextUnmatchedRows *= 100;
    +        // Report the row
    +        LOG.warn("" + unmatchedRows + " unmatched rows are found: " + line);
    --- End diff --
    
    Never mind. Printing which line is not matched will be much helpful.


---
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] tajo pull request #1046: TAJO-2179: Add a regular expression scanner and app...

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

    https://github.com/apache/tajo/pull/1046#discussion_r76532163
  
    --- Diff: tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/regex/RegexLineDeserializer.java ---
    @@ -0,0 +1,167 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.tajo.storage.regex;
    +
    +
    +import io.netty.buffer.ByteBuf;
    +import io.netty.util.CharsetUtil;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.tajo.catalog.Column;
    +import org.apache.tajo.catalog.Schema;
    +import org.apache.tajo.catalog.TableMeta;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.datum.NullDatum;
    +import org.apache.tajo.exception.InvalidTablePropertyException;
    +import org.apache.tajo.exception.TajoRuntimeException;
    +import org.apache.tajo.plan.util.PlannerUtil;
    +import org.apache.tajo.storage.FieldSerializerDeserializer;
    +import org.apache.tajo.storage.StorageConstants;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.storage.text.TextFieldSerializerDeserializer;
    +import org.apache.tajo.storage.text.TextLineDeserializer;
    +import org.apache.tajo.storage.text.TextLineParsingError;
    +import org.apache.tajo.storage.text.TextLineSerDe;
    +
    +import java.io.IOException;
    +import java.nio.charset.CharsetDecoder;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +public class RegexLineDeserializer extends TextLineDeserializer {
    +  private static final Log LOG = LogFactory.getLog(RegexLineDeserializer.class);
    +
    +  private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8);
    +  private FieldSerializerDeserializer fieldSerDer;
    +  private ByteBuf nullChars;
    +
    +  private int[] targetColumnIndexes;
    +  private String inputRegex;
    +  private Pattern inputPattern;
    +  // Number of rows not matching the regex
    +  private long unmatchedRows = 0;
    +  private long nextUnmatchedRows = 1;
    +  // Number of rows that match the regex but have missing groups.
    +  private long partialMatchedRows = 0;
    +  private long nextPartialMatchedRows = 1;
    +
    +  public RegexLineDeserializer(Schema schema, TableMeta meta, Column[] projected) {
    +    super(schema, meta);
    +    targetColumnIndexes = PlannerUtil.getTargetIds(schema, projected);
    +  }
    +
    +  @Override
    +  public void init() {
    +    fieldSerDer = new TextFieldSerializerDeserializer(meta);
    +    fieldSerDer.init(schema);
    +
    +    // Read the configuration parameters
    +    inputRegex = meta.getProperty(StorageConstants.TEXT_REGEX);
    +    boolean inputRegexIgnoreCase = "true".equalsIgnoreCase(
    +        meta.getProperty(StorageConstants.TEXT_REGEX_CASE_INSENSITIVE, "false"));
    +
    +    // Parse the configuration parameters
    +    if (inputRegex != null) {
    +      inputPattern = Pattern.compile(inputRegex, Pattern.DOTALL
    +          + (inputRegexIgnoreCase ? Pattern.CASE_INSENSITIVE : 0));
    +    } else {
    +      throw new TajoRuntimeException(new InvalidTablePropertyException(StorageConstants.TEXT_REGEX,
    +          "This table does not have serde property \"" + StorageConstants.TEXT_REGEX + "\"!"));
    +    }
    +
    +    if (nullChars != null) {
    +      nullChars.release();
    +    }
    +    nullChars = TextLineSerDe.getNullChars(meta);
    +  }
    +
    +
    +  @Override
    +  public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError {
    +
    +    if (lineBuf == null || targetColumnIndexes.length == 0) {
    +      return;
    +    }
    +
    +    String line = decoder.decode(lineBuf.nioBuffer(lineBuf.readerIndex(), lineBuf.readableBytes())).toString();
    +    int[] projection = targetColumnIndexes;
    +
    +    // Projection
    +    int currentTarget = 0;
    +    int currentIndex = 0;
    +    Matcher m = inputPattern.matcher(line);
    +
    +    if (!m.matches()) {
    +      unmatchedRows++;
    +      if (unmatchedRows >= nextUnmatchedRows) {
    +        nextUnmatchedRows *= 100;
    +        // Report the row
    +        LOG.warn("" + unmatchedRows + " unmatched rows are found: " + line);
    --- End diff --
    
    This also looks good, and I'm just curious why this log is printed repeatedly when 100 unmatched rows are found instead of just printing the total number of unmatched rows when the deserializer is released. Is it more helpful?


---
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] tajo pull request #1046: TAJO-2179: Add a regular expression scanner and app...

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

    https://github.com/apache/tajo/pull/1046


---
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] tajo issue #1046: TAJO-2179: Add a regular expression scanner and appender.

Posted by jihoonson <gi...@git.apache.org>.
Github user jihoonson commented on the issue:

    https://github.com/apache/tajo/pull/1046
  
    +1 looks good to me. Sorry for late review.


---
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.
---