You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by devtagare <gi...@git.apache.org> on 2016/03/15 15:43:23 UTC

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

GitHub user devtagare opened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/211

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

    Please review changes for Avro to Pojo conversion & back.
    
    Thanks,
    Dev

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

    $ git pull https://github.com/devtagare/incubator-apex-malhar APEXMALHAR-2011-2012

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

    https://github.com/apache/incubator-apex-malhar/pull/211.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 #211
    
----
commit 12af6b62ada24417fddb78b533d71d3cf6dc58a1
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-15T14:39:06Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

----


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57850900
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for reading Avro converted records
    + */
    +public class AvroRecordHelper
    +{
    +
    +  /**
    +   * Convert a passed String value to the given type for the key as per Schema
    +   */
    +  public static Object convertValueStringToAvroKeyType(Schema schema, String key, String value) throws ParseException
    +  {
    +    Type type = null;
    +
    +    if (schema.getField(key) != null) {
    +      type = schema.getField(key).schema().getType();
    +    } else {
    +      return value;
    +    }
    +
    +    Object convertedValue = null;
    +
    +    if (type == Type.UNION) {
    +      convertedValue = convertAndResolveUnionToPrimitive(schema, key, value);
    +    } else {
    +      convertedValue = convertValueToAvroPrimitive(type, key, value);
    +    }
    +
    +    return convertedValue;
    +
    +  }
    +
    +  private static Object convertValueToAvroPrimitive(Type type, String key, String value) throws ParseException
    +  {
    +    Object newValue = value;
    +    switch (type) {
    +      case BOOLEAN:
    +        newValue = Boolean.parseBoolean(value);
    +        break;
    +      case DOUBLE:
    +        newValue = Double.parseDouble(value);
    +        break;
    +      case FLOAT:
    +        newValue = Float.parseFloat(value);
    +        break;
    +      case INT:
    +        newValue = Integer.parseInt(value);
    +        break;
    +      case LONG:
    +        newValue = Long.parseLong(value);
    +        break;
    +      case BYTES:
    +        newValue = value.getBytes();
    +        break;
    +      case STRING:
    +        newValue = value;
    +        break;
    +      case NULL:
    +        newValue = null;
    +        break;
    +      default:
    +        newValue = value;
    +    }
    +    return newValue;
    +  }
    +
    +  private static Object convertAndResolveUnionToPrimitive(Schema schema, String key, String value) throws ParseException
    +  {
    +    Schema unionSchema = schema.getField(key).schema();
    +    List<Schema> types = unionSchema.getTypes();
    +    Object convertedValue = null;
    +    for (int i = 0; i < types.size(); i++) {
    +      try {
    +        if (types.get(i).getType() == Type.NULL) {
    +          if (value == null || value.equals("null")) {
    +            convertedValue = null;
    +            break;
    +          } else {
    +            continue;
    +          }
    +        }
    +        convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
    +      } catch (RuntimeException e) {
    +        LOG.error("Could not handle schema resolution" + e.getMessage());
    +        continue;
    +      }
    +      break;
    +    }
    +
    +    return convertedValue;
    +  }
    +
    +  static final Logger LOG = LoggerFactory.getLogger(AvroRecordHelper.class);
    --- End diff --
    
    private


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57934967
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    --- End diff --
    
    Done


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298564
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for Avro converted records
    + */
    +
    +public class AvroRecordHelper
    +{
    +
    +  /**
    +   * Convert a passed String value to the given type for the key as per Schema
    +   * 
    +   * @param schema
    +   * @param key
    +   * @param value
    +   * @return
    +   */
    +
    +  public static Object convertValueStringToAvroKeyType(Schema schema, String key, String value) throws ParseException
    +  {
    +    Type type = null;
    +
    +    if (schema.getField(key) != null) {
    +      type = schema.getField(key).schema().getType();
    +    } else {
    +      return value;
    +    }
    +
    +    Object convertedValue = null;
    +
    +    if (type == Type.UNION) {
    +      convertedValue = convertAndResolveUnionToPrimitive(schema, key, value);
    +    } else {
    +      convertedValue = convertValueToAvroPrimitive(type, key, value);
    +    }
    +
    +    return convertedValue;
    +
    +  }
    +
    +  private static Object convertValueToAvroPrimitive(Type type, String key, String value) throws ParseException
    +  {
    +    Object newValue = value;
    +    switch (type) {
    +      case BOOLEAN:
    +        newValue = Boolean.parseBoolean(value);
    +        break;
    +      case DOUBLE:
    +        newValue = Double.parseDouble(value);
    +        break;
    +      case FLOAT:
    +        newValue = Float.parseFloat(value);
    +        break;
    +      case INT:
    +        newValue = Integer.parseInt(value);
    +        break;
    +      case LONG:
    +        newValue = Long.parseLong(value);
    +        break;
    +      case BYTES:
    +        newValue = value.getBytes();
    +        break;
    +      case STRING:
    +        newValue = value;
    +        break;
    +      case NULL:
    +        newValue = null;
    +        break;
    +      default:
    +        newValue = value;
    +    }
    +    return newValue;
    +  }
    +
    +  private static Object convertAndResolveUnionToPrimitive(Schema schema, String key, String value) throws ParseException
    +  {
    +    Schema unionSchema = schema.getField(key).schema();
    +    List<Schema> types = unionSchema.getTypes();
    +    Object convertedValue = null;
    +    for (int i = 0; i < types.size(); i++) {
    +      try {
    +        if (types.get(i).getType() == Type.NULL) {
    +          if (value == null || value.equals("null")) {
    +            convertedValue = null;
    +            break;
    +          } else {
    +            continue;
    +          }
    +        }
    +        convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
    --- End diff --
    
    Can you directly return from here and skip return on line 122? code would be more readable.


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

Re: [GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

Posted by Devendra Tagare <de...@datatorrent.com>.
Failure in reading a record should not cause a failure to read the entire
file.

On Tue, Mar 29, 2016 at 10:18 AM, chinmaykolhatkar <gi...@git.apache.org>
wrote:

> Github user chinmaykolhatkar commented on a diff in the pull request:
>
>
> https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57762245
>
>     --- Diff:
> contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java
> ---
>     @@ -0,0 +1,159 @@
>     +/**
>     + * 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 com.datatorrent.contrib.avro;
>     +
>     +import java.io.IOException;
>     +import java.io.InputStream;
>     +
>     +import org.slf4j.Logger;
>     +import org.slf4j.LoggerFactory;
>     +
>     +import org.apache.avro.AvroRuntimeException;
>     +import org.apache.avro.file.DataFileStream;
>     +import org.apache.avro.generic.GenericDatumReader;
>     +import org.apache.avro.generic.GenericRecord;
>     +import org.apache.avro.io.DatumReader;
>     +import org.apache.hadoop.classification.InterfaceStability;
>     +import org.apache.hadoop.fs.Path;
>     +
>     +import com.datatorrent.api.AutoMetric;
>     +import com.datatorrent.api.DefaultOutputPort;
>     +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
>     +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
>     +
>     +/**
>     + * <p>
>     + * Avro File Input Operator
>     + * </p>
>     + * A specific implementation of the AbstractFileInputOperator to read
> Avro
>     + * container files.<br>
>     + * No need to provide schema,its inferred from the file<br>
>     + * Users can add the {@link
> IdempotentStorageManager.FSIdempotentStorageManager}
>     + * to ensure exactly once semantics with a HDFS backed WAL.
>     + *
>     + * @displayName AvroFileInputOperator
>     + * @category Input
>     + * @tags fs, file,avro, input operator
>     + * @since 3.3.0
>     + */
>     +@InterfaceStability.Evolving
>     +public class AvroFileInputOperator extends
> AbstractFileInputOperator<GenericRecord>
>     +{
>     +
>     +  private transient long offset = 0L;
>     +
>     +  @AutoMetric
>     +  int recordCnt = 0;
>     +
>     +  @AutoMetric
>     +  int errorCnt = 0;
>     +
>     +  private transient DataFileStream<GenericRecord> avroDataStream;
>     +  private transient GenericRecord record = null;
>     +
>     +  public final transient DefaultOutputPort<GenericRecord> output =
> new DefaultOutputPort<GenericRecord>();
>     +
>     +  @OutputPortFieldAnnotation(optional = true)
>     +  public final transient DefaultOutputPort<String> completedFilesPort
> = new DefaultOutputPort<String>();
>     +
>     +  @OutputPortFieldAnnotation(optional = true)
>     +  public final transient DefaultOutputPort<String> errorRecordsPort =
> new DefaultOutputPort<String>();
>     +
>     +  @Override
>     +  protected InputStream openFile(Path path) throws IOException
>     +  {
>     +    InputStream is = super.openFile(path);
>     +    if (is != null) {
>     +      try {
>     +        DatumReader<GenericRecord> datumReader = new
> GenericDatumReader<GenericRecord>();
>     +        avroDataStream = new DataFileStream<GenericRecord>(is,
> datumReader);
>     +        datumReader.setSchema(avroDataStream.getSchema());
>     +        return is;
>     +      } catch (NullPointerException npe) {
>     +        LOG.error("Schemaless file - " + npe.getMessage());
>     +      }
>     +    }
>     +    return null;
>     --- End diff --
>
>     What happens when this method returns null?
>     You take care of following cases:
>     1. How is null handled?
>     2. Is there a point in returning null or probably just throw this as a
> RuntimeException if the situation is not recoverable?
>
>
> ---
> 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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57762245
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +        return is;
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file - " + npe.getMessage());
    +      }
    +    }
    +    return null;
    --- End diff --
    
    What happens when this method returns null?
    You take care of following cases:
    1. How is null handled?
    2. Is there a point in returning null or probably just throw this as a RuntimeException if the situation is not recoverable?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57934571
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Changed the variable names.Not sure about marking autometrics as private.Can you give a rationale & point to the places where its marked private.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57762011
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    --- End diff --
    
    Can you please add the method javadoc for openFile and describe what it does?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298364
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +
    +          }
    +
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +    }
    +
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   */
    +
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +
    +    }
    +
    +    return sb.substring(0, sb.length() - 1).toString();
    +
    +  }
    +
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +
    +    columnFieldSetters = Lists.newArrayList();
    +
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +
    +    if (getFieldInfos().isEmpty() || getGenericRecordToPOJOFieldsMapping() == null) {
    +      setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    --- End diff --
    
    Is this working?
    
    cls variable is set in output Port's setup method which gets called after this method. So getCls would return null here?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845253
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    --- End diff --
    
    Can this be made private?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846388
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    +      errorCnt++;
    +    }
    +
    +    if (record != null) {
    +      output.emit(record);
    +      recordCnt++;
    +    } else {
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a generic record mapping the POJO fields to provided schema
    +   * 
    +   * @return Generic Record
    +   */
    +  private GenericRecord getGenericRecord(Object tuple) throws Exception
    +  {
    +    int writeErrorCount = 0;
    +    GenericRecord rec = new GenericData.Record(getSchema());
    +
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        rec.put(columnNames.get(i).name(), AvroRecordHelper.convertValueStringToAvroKeyType(getSchema(),
    +            columnNames.get(i).name(), keyMethodMap.get(i).get(tuple).toString()));
    +      } catch (AvroRuntimeException e) {
    +        LOG.error("Could not set Field [" + columnNames.get(i).name() + "] in the generic record");
    +        fieldErrorCnt++;
    +      } catch (Exception e) {
    +        LOG.error("Parse Exception");
    +        fieldErrorCnt++;
    +        writeErrorCount++;
    +      }
    +    }
    +
    +    if (columnNames.size() == writeErrorCount) {
    +      return null;
    +    } else {
    +      return rec;
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    recordCnt = 0;
    --- End diff --
    
    @chandnisingh , Can you please check if this is right place to reset autometrics variable OR should this be done in beginWindow.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57761587
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    --- End diff --
    
    No need to set output port as optional... Its optional by default.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56291316
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    --- End diff --
    
    Does this need to be public? Is this used from outside of this class?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57965227
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    --- End diff --
    
    its on tuple.get(afi.fieldInfo.getColumnName()) not on new String


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r58024050
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    --- End diff --
    
    Well yes it should be there too. But considering the review is for this file I suggest to put it here because the default functionality is overridden or extended.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57850600
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for reading Avro converted records
    + */
    +public class AvroRecordHelper
    --- End diff --
    
    I see that this class is used only in PojoToAvro.getGenericRecord. Please correct me if I'm wrong.
    In that case, can you move this to that class itself as a private method?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298417
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +
    +          }
    +
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +    }
    +
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   */
    +
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +
    +    }
    +
    +    return sb.substring(0, sb.length() - 1).toString();
    +
    +  }
    +
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +
    +    columnFieldSetters = Lists.newArrayList();
    +
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +
    +    if (getFieldInfos().isEmpty() || getGenericRecordToPOJOFieldsMapping() == null) {
    +      setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +    } else {
    +      setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +    }
    +
    +    initColumnFieldSetters(getFieldInfos());
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    // TODO Auto-generated method stub
    +
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    // TODO Auto-generated method stub
    --- End diff --
    
    I think you need to reset your autometric variables here.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56290631
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for Avro converted records
    + */
    +
    +public class AvroRecordHelper
    +{
    +
    +  /**
    +   * Convert a passed String value to the given type for the key as per Schema
    +   * 
    +   * @param schema
    +   * @param key
    +   * @param value
    +   * @return
    +   */
    +
    --- End diff --
    
    Please remove spaces between start of function line and comment. The comment may not get picked up for javadocs.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57766614
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +        return is;
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file - " + npe.getMessage());
    --- End diff --
    
    Done


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287492
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/SimpleOrder.java ---
    @@ -0,0 +1,93 @@
    +/**
    --- End diff --
    
    It is possible to move it as a static class inside the test file, than creating a separate file?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297756
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    --- End diff --
    
    I'm not sure if this would work if the output POJO has wrapper classes.
    Can you please add such a test?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57766719
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    --- End diff --
    
    Done



---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299388
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroReaderWriterTest.java ---
    @@ -0,0 +1,219 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.ListIterator;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +
    +import junit.framework.Assert;
    +
    +@SuppressWarnings("deprecation")
    +public class AvroReaderWriterTest {
    +
    +	private AvroToPojo reader = null;
    +	private PojoToAvro writer = null;
    +
    +	public static final String schemaFile = "{\"namespace\":\"abc\",\"type\":\"record\",\"doc\":\"Order schema\",\"name\":\"Order\",\"fields\":[{\"name\":\"orderId\",\"type\": \"long\"},{\"name\":\"customerId\",\"type\": \"int\"},{\"name\":\"total\",\"type\": \"double\"},{\"name\":\"customerName\",\"type\": \"string\"}]}";
    +
    +	public static final String fieldInfoInitMap = "orderId:orderId:LONG,customerId:customerId:INTEGER,customerName:customerName:STRING,total:total:DOUBLE";
    +
    +	private Class<?> className = SimpleOrder.class;
    +
    +	private List<GenericRecord> recordList = null;
    +
    +	public AvroReaderWriterTest() {
    +
    +	}
    +
    +	@Before
    +	public void initializeReaderAndWriter() {
    +		reader = new AvroToPojo();
    +		writer = new PojoToAvro();
    +
    +		recordList = new LinkedList<GenericRecord>();
    +
    +		initializeWriter();
    +		initializeReader();
    +
    +	}
    +
    +	public void initializeWriter() {
    +
    +		getWriter().setSchemaString(schemaFile);
    +
    +		try {
    +			getWriter().parseSchema();
    +		} catch (IOException e) {
    +			LOG.error("Error in parsing");
    +		}
    +
    +		getWriter().setCls(className);
    +
    +		getWriter().initializeColumnMap(writer.getSchema());
    +
    +	}
    +
    +	public void createReaderInput() {
    +		int cnt = 3;
    +
    +		while (cnt > 0) {
    +
    +			GenericRecord rec = new GenericData.Record(reader.getSchema());
    +			rec.put("orderId", cnt * 1);
    +			rec.put("customerId", cnt * 2);
    +			rec.put("total", cnt * 1.5);
    +			rec.put("customerName", "*" + cnt + "*");
    +			cnt--;
    +			recordList.add(rec);
    +
    +		}
    +	}
    +
    +	public void initializeReader() {
    +		reader.setSchemaString(schemaFile);
    +
    +		try {
    +			getReader().parseSchema();
    +		} catch (IOException e) {
    +			LOG.error("Exception in parsing schema," + e.getMessage());
    +		}
    +
    +		getReader().setCls(className);
    +
    +		/**
    +		 * To set the field mapping we use comma separated list of
    +		 * fieldInPojo:FieldInGenericRecord:DataType This can be set as a part
    +		 * of the operator initialization phase on the fieldInfoInitMap string
    +		 * If this is not given the reflection is used only once to generate
    +		 * this map.
    +		 */
    +
    +		if (fieldInfoInitMap.isEmpty() || getReader().getGenericRecordToPOJOFieldsMapping() == null) {
    +			getReader().setFieldInfos(
    +					getReader().createFieldInfoMap(getReader().generateFieldInfoInputs(SimpleOrder.class)));
    +		} else {
    +			getReader().setFieldInfos(getReader().createFieldInfoMap(fieldInfoInitMap));
    +		}
    +
    +		getReader().initColumnFieldSetters(getReader().getFieldInfos());
    +
    +		createReaderInput();
    +
    +	}
    +
    +	public AvroToPojo getReader() {
    +		return reader;
    +	}
    +
    +	public void setReader(AvroToPojo reader) {
    +		this.reader = reader;
    +	}
    +
    +	public PojoToAvro getWriter() {
    +		return writer;
    +	}
    +
    +	public void setWriter(PojoToAvro writer) {
    +		this.writer = writer;
    +	}
    +
    +	@Test
    +	public void testWriter() {
    --- End diff --
    
    You need to make operator go through proper lifecycle in unit tests.
    op.setup()
    op.beingWindow(x)
    op.port.put(<tuple>)
    op.endWindow();
    op.teardown()


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846114
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    --- End diff --
    
    Please don't hide the exception callstack.. Please log 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845290
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      columnFieldSetters = Lists.newArrayList();
    +
    +      if (getGenericRecordToPOJOFieldsMapping() == null) {
    +        setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +      } else {
    +        setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +      }
    +
    +      initColumnFieldSetters(getFieldInfos());
    +      initializeActiveFieldSetters();
    +    }
    +  };
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  /**
    +   * Class that maps fieldInfo to its getters or setters
    +   */
    +  protected static class ActiveFieldInfo
    +  {
    +    final FieldInfo fieldInfo;
    +    Object setterOrGetter;
    +
    +    ActiveFieldInfo(FieldInfo fieldInfo)
    +    {
    +      this.fieldInfo = fieldInfo;
    +    }
    +  }
    +
    +  /**
    +   * A list of {@link FieldInfo}s where each item maps a column name to a pojo
    +   * field name.
    +   */
    +  private List<FieldInfo> getFieldInfos()
    +  {
    +    return fieldInfos;
    +  }
    +
    +  /**
    +   * Add the Active Fields to the columnFieldSetters {@link ActiveFieldInfo}s
    +   */
    +  public void initColumnFieldSetters(List<FieldInfo> fieldInfos)
    --- End diff --
    
    Can this be private?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844890
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    --- End diff --
    
    Same assignment done twice.. Can you please remove one?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298968
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    +  {
    +    columnNames = schema.getFields();
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, columnNames.get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = true, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +
    +      GenericRecord record = null;
    --- End diff --
    
    Please create a protected process method at operator class level and call that method from this process method.
    This is to ensure that one can extend processing of tuple. 


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56300024
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    --- End diff --
    
    Can you rename the variables to have meaningful names e.g. afi can be fieldInfo or activeFieldInfo?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299066
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    +  {
    +    columnNames = schema.getFields();
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, columnNames.get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = true, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +
    +      GenericRecord record = null;
    +
    +      try {
    +        record = getGenericRecord(tuple);
    +      } catch (Exception e) {
    +        LOG.error("Exception in parsing record");
    +        errorCnt++;
    +      }
    +
    +      if (record != null) {
    +        output.emit(record);
    +        recordCnt++;
    +      } else {
    +        errorCnt++;
    +      }
    +
    +    }
    +
    +  };
    +
    +  /**
    +   * @param -Object
    +   *          Returns a generic record mapping the POJO fields to provided
    +   *          schema
    +   */
    +
    +  public GenericRecord getGenericRecord(Object tuple) throws Exception
    --- End diff --
    
    Why public?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845171
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    --- End diff --
    
    Is this method expected to be used outside of the class? If not please make it private.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844708
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    --- End diff --
    
    cls is unused.. Please remove 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57850367
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for reading Avro converted records
    + */
    +public class AvroRecordHelper
    +{
    +
    +  /**
    +   * Convert a passed String value to the given type for the key as per Schema
    +   */
    +  public static Object convertValueStringToAvroKeyType(Schema schema, String key, String value) throws ParseException
    +  {
    +    Type type = null;
    +
    +    if (schema.getField(key) != null) {
    --- End diff --
    
    Can scheme be null here?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298880
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    +  {
    +    columnNames = schema.getFields();
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, columnNames.get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = true, schemaRequired = true)
    --- End diff --
    
    Not sure if I understand the purpose of input port being optional.
    
    If this is the only input port and optional to connect, why would anyone add this operator?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298721
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    --- End diff --
    
    Why public?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846459
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    +      errorCnt++;
    +    }
    +
    +    if (record != null) {
    +      output.emit(record);
    +      recordCnt++;
    +    } else {
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a generic record mapping the POJO fields to provided schema
    +   * 
    +   * @return Generic Record
    +   */
    +  private GenericRecord getGenericRecord(Object tuple) throws Exception
    +  {
    +    int writeErrorCount = 0;
    +    GenericRecord rec = new GenericData.Record(getSchema());
    +
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        rec.put(columnNames.get(i).name(), AvroRecordHelper.convertValueStringToAvroKeyType(getSchema(),
    +            columnNames.get(i).name(), keyMethodMap.get(i).get(tuple).toString()));
    +      } catch (AvroRuntimeException e) {
    +        LOG.error("Could not set Field [" + columnNames.get(i).name() + "] in the generic record");
    +        fieldErrorCnt++;
    +      } catch (Exception e) {
    +        LOG.error("Parse Exception");
    +        fieldErrorCnt++;
    +        writeErrorCount++;
    +      }
    +    }
    +
    +    if (columnNames.size() == writeErrorCount) {
    +      return null;
    +    } else {
    +      return rec;
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    recordCnt = 0;
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +  }
    +
    +  public static final Logger LOG = LoggerFactory.getLogger(PojoToAvro.class);
    --- End diff --
    
    Should be private.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845414
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Can you make this private?
    Same for other AutoMetrics too. 


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846305
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    +      errorCnt++;
    +    }
    +
    +    if (record != null) {
    +      output.emit(record);
    +      recordCnt++;
    +    } else {
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a generic record mapping the POJO fields to provided schema
    +   * 
    +   * @return Generic Record
    +   */
    +  private GenericRecord getGenericRecord(Object tuple) throws Exception
    +  {
    +    int writeErrorCount = 0;
    +    GenericRecord rec = new GenericData.Record(getSchema());
    +
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        rec.put(columnNames.get(i).name(), AvroRecordHelper.convertValueStringToAvroKeyType(getSchema(),
    +            columnNames.get(i).name(), keyMethodMap.get(i).get(tuple).toString()));
    +      } catch (AvroRuntimeException e) {
    +        LOG.error("Could not set Field [" + columnNames.get(i).name() + "] in the generic record");
    +        fieldErrorCnt++;
    +      } catch (Exception e) {
    +        LOG.error("Parse Exception");
    --- End diff --
    
    Please log exception.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r58023007
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    --- End diff --
    
    Ok then new String() is not required as toString() returns String object itself.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57763599
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    --- End diff --
    
    There isn't  any.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57936247
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    +
    +    try {
    +      if (avroDataStream != null && avroDataStream.hasNext()) {
    +        offset++;
    +
    +        record = avroDataStream.next();
    +        recordCnt++;
    +        return record;
    +      }
    +    } catch (AvroRuntimeException are) {
    +      LOG.error("Exception in parsing record for file - " + super.currentFile + " at offset - " + offset, are);
    +      if (errorRecordsPort.isConnected()) {
    +        errorRecordsPort.emit("FileName:" + super.currentFile + ", Offset:" + offset);
    +      }
    +      errorCnt++;
    +      throw new AvroRuntimeException(are);
    +    }
    +    return record;
    +  }
    +
    +  @Override
    +  protected void closeFile(InputStream is) throws IOException
    +  {
    +    String fileName = super.currentFile;
    +    super.closeFile(is);
    +    if (avroDataStream != null) {
    +      avroDataStream.close();
    +    }
    +    if (completedFilesPort.isConnected()) {
    +      completedFilesPort.emit(fileName);
    +    }
    +    offset = 0;
    +  }
    +
    +  @Override
    +  protected void emit(GenericRecord tuple)
    +  {
    +    if (tuple != null) {
    +      output.emit(tuple);
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    --- End diff --
    
    Had a chat with @chandnisingh. Marking them private & marking @VisibleForTesting
     Add a line note



---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845229
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    --- End diff --
    
    This is very picky... But you already are working with StringBuilder.
    you can just do sb.append(...).append(...)...


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844263
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      columnFieldSetters = Lists.newArrayList();
    +
    +      if (getGenericRecordToPOJOFieldsMapping() == null) {
    +        setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +      } else {
    +        setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +      }
    +
    +      initColumnFieldSetters(getFieldInfos());
    +      initializeActiveFieldSetters();
    +    }
    +  };
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    --- End diff --
    
    @chandnisingh Can you please comment if this is ok place to reset autometric variables considering when they're picked up? And what is the best practice?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57761346
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +        return is;
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file - " + npe.getMessage());
    --- End diff --
    
    Please don't hide any exception with logs... You could log the exception as well which would show callstack for debugging purpose...


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57841688
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    --- End diff --
    
    Why have another level method redirection if nothing else is in 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

Posted by devtagare <gi...@git.apache.org>.
GitHub user devtagare reopened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/211

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

    Please review changes for Avro to Pojo conversion & back.
    
    Thanks,
    Dev

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

    $ git pull https://github.com/devtagare/incubator-apex-malhar APEXMALHAR-2011-2012

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

    https://github.com/apache/incubator-apex-malhar/pull/211.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 #211
    
----
commit 12af6b62ada24417fddb78b533d71d3cf6dc58a1
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-15T14:39:06Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

commit 6f9f301cad64577556991898966b9bd232d8e7db
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T13:29:43Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Review comments

commit 57600a4dfb727265644d093bd2cdeec46e917273
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T14:41:18Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Removed unused spaces & variables

commit eff2510a77accea89c363de538d205267ba56ef5
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T00:09:44Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 4ea1f8421e5e49d268b3ad0523febd673eb65762
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:30:25Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 9b785e1281dc9b645668d48959fee64cb4f871c0
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:37:46Z

    APEXMALHAR-2011-2012 Avro File Input Operator - Added documentation

commit 4b2b80e8a0cd2fa4f9a1ca3b768f858657f572e3
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T21:10:49Z

    APEXMALHAR-2011-2012 Avro File Input Operator - removed default constructor

----


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57763061
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    --- End diff --
    
    We should add that DOC to the AbstractFileInputOperator as well.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57841449
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    --- End diff --
    
    I don't see "record" variable being used anywhere else other than this method.
    Can you make this in local scope instead of class level?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844401
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      columnFieldSetters = Lists.newArrayList();
    +
    +      if (getGenericRecordToPOJOFieldsMapping() == null) {
    +        setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +      } else {
    +        setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +      }
    +
    +      initColumnFieldSetters(getFieldInfos());
    +      initializeActiveFieldSetters();
    +    }
    +  };
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    --- End diff --
    
    How is user expected to set cls from properties file if that's expected? Maybe take qualified class name and revolve 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57761130
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    --- End diff --
    
    Please don't add @since tag in code.. It would be added automatically during release process.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297389
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    --- End diff --
    
    Yes please. Doing this everytime is very costly.



---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299275
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroReaderWriterTest.java ---
    @@ -0,0 +1,219 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.ListIterator;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +
    +import junit.framework.Assert;
    +
    +@SuppressWarnings("deprecation")
    +public class AvroReaderWriterTest {
    +
    +	private AvroToPojo reader = null;
    +	private PojoToAvro writer = null;
    +
    +	public static final String schemaFile = "{\"namespace\":\"abc\",\"type\":\"record\",\"doc\":\"Order schema\",\"name\":\"Order\",\"fields\":[{\"name\":\"orderId\",\"type\": \"long\"},{\"name\":\"customerId\",\"type\": \"int\"},{\"name\":\"total\",\"type\": \"double\"},{\"name\":\"customerName\",\"type\": \"string\"}]}";
    +
    +	public static final String fieldInfoInitMap = "orderId:orderId:LONG,customerId:customerId:INTEGER,customerName:customerName:STRING,total:total:DOUBLE";
    +
    +	private Class<?> className = SimpleOrder.class;
    +
    +	private List<GenericRecord> recordList = null;
    +
    +	public AvroReaderWriterTest() {
    +
    +	}
    +
    +	@Before
    +	public void initializeReaderAndWriter() {
    +		reader = new AvroToPojo();
    +		writer = new PojoToAvro();
    +
    +		recordList = new LinkedList<GenericRecord>();
    +
    +		initializeWriter();
    +		initializeReader();
    +
    +	}
    +
    +	public void initializeWriter() {
    +
    +		getWriter().setSchemaString(schemaFile);
    +
    +		try {
    +			getWriter().parseSchema();
    +		} catch (IOException e) {
    +			LOG.error("Error in parsing");
    +		}
    +
    +		getWriter().setCls(className);
    +
    +		getWriter().initializeColumnMap(writer.getSchema());
    +
    +	}
    +
    +	public void createReaderInput() {
    +		int cnt = 3;
    +
    +		while (cnt > 0) {
    +
    --- End diff --
    
    Indentation is off.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845905
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    --- End diff --
    
    No need fo explicitly mentioning optional=false for input port. That's default.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57842448
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    +
    +    try {
    +      if (avroDataStream != null && avroDataStream.hasNext()) {
    +        offset++;
    +
    +        record = avroDataStream.next();
    +        recordCnt++;
    +        return record;
    +      }
    +    } catch (AvroRuntimeException are) {
    +      LOG.error("Exception in parsing record for file - " + super.currentFile + " at offset - " + offset, are);
    +      if (errorRecordsPort.isConnected()) {
    +        errorRecordsPort.emit("FileName:" + super.currentFile + ", Offset:" + offset);
    +      }
    +      errorCnt++;
    +      throw new AvroRuntimeException(are);
    +    }
    +    return record;
    +  }
    +
    +  @Override
    +  protected void closeFile(InputStream is) throws IOException
    +  {
    +    String fileName = super.currentFile;
    +    super.closeFile(is);
    +    if (avroDataStream != null) {
    +      avroDataStream.close();
    +    }
    +    if (completedFilesPort.isConnected()) {
    +      completedFilesPort.emit(fileName);
    +    }
    +    offset = 0;
    +  }
    +
    +  @Override
    +  protected void emit(GenericRecord tuple)
    +  {
    +    if (tuple != null) {
    +      output.emit(tuple);
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    recordCnt = 0;
    +  }
    +
    +  final Logger LOG = LoggerFactory.getLogger(AvroFileInputOperator.class);
    --- End diff --
    
    "private static"


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57851509
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroToPojoTest.java ---
    @@ -0,0 +1,294 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.util.List;
    +import java.util.ListIterator;
    +
    +import org.junit.Assert;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.TestWatcher;
    +import org.junit.runner.Description;
    +import org.python.google.common.collect.Lists;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +
    +import com.datatorrent.api.Attribute;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.helper.TestPortContext;
    +import com.datatorrent.lib.testbench.CollectorTestSink;
    +
    +public class AvroToPojoTest
    --- End diff --
    
    Can you also add application test for this to check if any serialization issues exist?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57842432
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    +
    +    try {
    +      if (avroDataStream != null && avroDataStream.hasNext()) {
    +        offset++;
    +
    +        record = avroDataStream.next();
    +        recordCnt++;
    +        return record;
    +      }
    +    } catch (AvroRuntimeException are) {
    +      LOG.error("Exception in parsing record for file - " + super.currentFile + " at offset - " + offset, are);
    +      if (errorRecordsPort.isConnected()) {
    +        errorRecordsPort.emit("FileName:" + super.currentFile + ", Offset:" + offset);
    +      }
    +      errorCnt++;
    +      throw new AvroRuntimeException(are);
    +    }
    +    return record;
    +  }
    +
    +  @Override
    +  protected void closeFile(InputStream is) throws IOException
    +  {
    +    String fileName = super.currentFile;
    +    super.closeFile(is);
    +    if (avroDataStream != null) {
    +      avroDataStream.close();
    +    }
    +    if (completedFilesPort.isConnected()) {
    +      completedFilesPort.emit(fileName);
    +    }
    +    offset = 0;
    +  }
    +
    +  @Override
    +  protected void emit(GenericRecord tuple)
    +  {
    +    if (tuple != null) {
    +      output.emit(tuple);
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    --- End diff --
    
    Can you please check with @chandnisingh about what is the best practice to reset autometric variables? In beginWindow or endWindow?
    
    If values of AutoMetric variables are been picked up after endWindow is called, the values will always be 0.



---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

Posted by devtagare <gi...@git.apache.org>.
GitHub user devtagare reopened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/211

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

    Please review changes for Avro to Pojo conversion & back.
    
    Thanks,
    Dev

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

    $ git pull https://github.com/devtagare/incubator-apex-malhar APEXMALHAR-2011-2012

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

    https://github.com/apache/incubator-apex-malhar/pull/211.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 #211
    
----
commit 12af6b62ada24417fddb78b533d71d3cf6dc58a1
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-15T14:39:06Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

commit 6f9f301cad64577556991898966b9bd232d8e7db
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T13:29:43Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Review comments

commit 57600a4dfb727265644d093bd2cdeec46e917273
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T14:41:18Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Removed unused spaces & variables

commit eff2510a77accea89c363de538d205267ba56ef5
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T00:09:44Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 4ea1f8421e5e49d268b3ad0523febd673eb65762
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:30:25Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 9b785e1281dc9b645668d48959fee64cb4f871c0
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:37:46Z

    APEXMALHAR-2011-2012 Avro File Input Operator - Added documentation

commit 4b2b80e8a0cd2fa4f9a1ca3b768f858657f572e3
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T21:10:49Z

    APEXMALHAR-2011-2012 Avro File Input Operator - removed default constructor

----


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844648
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    --- End diff --
    
    Should there be a error port where GenericRecord is emitted in case of any failure?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57762934
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +        return is;
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file - " + npe.getMessage());
    +      }
    +    }
    +    return null;
    --- End diff --
    
    Failure to read a single record should not cause a failure to read the rest of the records from the data stream.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297650
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    --- End diff --
    
    CHeckstyle is off here.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56291019
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    --- End diff --
    
    Can you add javadoc for setter methods?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56624094
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -114,6 +107,7 @@ private static Object convertAndResolveUnionToPrimitive(Schema schema, String ke
             }
             convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
           } catch (RuntimeException e) {
    +        LOG.error("Could not handle schema resolution" + e.getMessage());
    --- End diff --
    
    Should this not actually throw to the caller instead of hiding here?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56291281
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    --- End diff --
    
    javadoc for Setter method 


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57934935
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    +
    +    try {
    +      if (avroDataStream != null && avroDataStream.hasNext()) {
    +        offset++;
    +
    +        record = avroDataStream.next();
    +        recordCnt++;
    +        return record;
    +      }
    +    } catch (AvroRuntimeException are) {
    +      LOG.error("Exception in parsing record for file - " + super.currentFile + " at offset - " + offset, are);
    +      if (errorRecordsPort.isConnected()) {
    +        errorRecordsPort.emit("FileName:" + super.currentFile + ", Offset:" + offset);
    +      }
    +      errorCnt++;
    +      throw new AvroRuntimeException(are);
    +    }
    +    return record;
    +  }
    +
    +  @Override
    +  protected void closeFile(InputStream is) throws IOException
    +  {
    +    String fileName = super.currentFile;
    +    super.closeFile(is);
    +    if (avroDataStream != null) {
    +      avroDataStream.close();
    +    }
    +    if (completedFilesPort.isConnected()) {
    +      completedFilesPort.emit(fileName);
    +    }
    +    offset = 0;
    +  }
    +
    +  @Override
    +  protected void emit(GenericRecord tuple)
    +  {
    +    if (tuple != null) {
    +      output.emit(tuple);
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    --- End diff --
    
    Noticed that autometric variables are not marked private in several operators.see CsvParser, JsonParser. They will not be accessible in the test class unless getters are provided.
    Has the usage changed since they were first implemented ?
    beginWindow seems a better place for resetting the values, moving it there.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298529
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +
    +          }
    +
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +    }
    +
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   */
    +
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +
    +    }
    +
    +    return sb.substring(0, sb.length() - 1).toString();
    +
    +  }
    +
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +
    +    columnFieldSetters = Lists.newArrayList();
    +
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +
    +    if (getFieldInfos().isEmpty() || getGenericRecordToPOJOFieldsMapping() == null) {
    +      setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +    } else {
    +      setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +    }
    +
    +    initColumnFieldSetters(getFieldInfos());
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    // TODO Auto-generated method stub
    +
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    // TODO Auto-generated method stub
    +
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    --- End diff --
    
    Why is this externall required variables?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57761540
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    --- End diff --
    
    Is it possible to utilize output port from AbstractFileInputOperator class instead of creating a new one?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57970394
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for reading Avro converted records
    + */
    +public class AvroRecordHelper
    --- End diff --
    
    This can be used by other classes for resolving avro schema.Its expected to be used as a helper to get values associated with all Avro types both primitive & non-primitive.A user can use it directly as a part of this operator to resolve 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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299147
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    +  {
    +    columnNames = schema.getFields();
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, columnNames.get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = true, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +
    +      GenericRecord record = null;
    +
    +      try {
    +        record = getGenericRecord(tuple);
    +      } catch (Exception e) {
    +        LOG.error("Exception in parsing record");
    +        errorCnt++;
    +      }
    +
    +      if (record != null) {
    +        output.emit(record);
    +        recordCnt++;
    +      } else {
    +        errorCnt++;
    +      }
    +
    +    }
    +
    +  };
    +
    +  /**
    +   * @param -Object
    +   *          Returns a generic record mapping the POJO fields to provided
    +   *          schema
    +   */
    +
    +  public GenericRecord getGenericRecord(Object tuple) throws Exception
    +  {
    +
    +    GenericRecord rec = new GenericData.Record(getSchema());
    +
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        rec.put(columnNames.get(i).name(), AvroRecordHelper.convertValueStringToAvroKeyType(getSchema(),
    +            columnNames.get(i).name(), keyMethodMap.get(i).get(tuple).toString()));
    +      } catch (AvroRuntimeException e) {
    +        LOG.error("Could not set Field [" + columnNames.get(i).name() + "] in the generic record");
    +        fieldErrorCnt++;
    +      } catch (Exception e) {
    +        LOG.error("Parse Exception");
    +        fieldErrorCnt++;
    +      }
    +    }
    +
    +    return rec;
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    // TODO Auto-generated method stub
    --- End diff --
    
    Please remove this comment. I suggest to extend the class BaseOperator class instead of implementing Operator interface in which case you would not need to have any of these empty method.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56290931
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    --- End diff --
    
    Please remove the space.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287478
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroReaderWriterTest.java ---
    @@ -0,0 +1,219 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    --- End diff --
    
    formatting is off for this file.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57841319
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    --- End diff --
    
    Suggesting to remove the try/catch.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845794
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    --- End diff --
    
    Please remove @since... This will be added during release process.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299440
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/Order.java ---
    @@ -0,0 +1,49 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.Serializable;
    +
    +public class Order implements Serializable
    --- End diff --
    
    Class level comments are required. This causes problem otherwise during release cycle.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298492
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    --- End diff --
    
    Yes.Thats the use case.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844328
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      columnFieldSetters = Lists.newArrayList();
    +
    +      if (getGenericRecordToPOJOFieldsMapping() == null) {
    +        setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +      } else {
    +        setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +      }
    +
    +      initColumnFieldSetters(getFieldInfos());
    +      initializeActiveFieldSetters();
    +    }
    +  };
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public Class<?> getCls()
    --- End diff --
    
    Is it expected by user to set cls? If not, can you please remove this? If yes, can you please give a more intuitive name than cls? 


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57843629
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Can you give a complete name like recordCount and errorCount?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r58023444
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -114,6 +107,7 @@ private static Object convertAndResolveUnionToPrimitive(Schema schema, String ke
             }
             convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
           } catch (RuntimeException e) {
    +        LOG.error("Could not handle schema resolution" + e.getMessage());
    --- End diff --
    
    Ok.. Please log the complete exception then instead of just e.getMessage()..


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57761599
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  @OutputPortFieldAnnotation(optional = true)
    --- End diff --
    
    No need to set output port as optional... Its optional by default.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287510
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    --- End diff --
    
    Is the schema serializable by Kryo, It does not have default constructor. What is the different between schemaFile and schema? Can schema be created from schemaFile? If yes you can make schema as transient and initialize it in the setup.



---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57842179
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    --- End diff --
    
    Can you please add a line saying the GenericRecord is emitted from this operator?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846258
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    +      errorCnt++;
    +    }
    +
    +    if (record != null) {
    +      output.emit(record);
    +      recordCnt++;
    +    } else {
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a generic record mapping the POJO fields to provided schema
    +   * 
    +   * @return Generic Record
    +   */
    +  private GenericRecord getGenericRecord(Object tuple) throws Exception
    +  {
    +    int writeErrorCount = 0;
    +    GenericRecord rec = new GenericData.Record(getSchema());
    +
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        rec.put(columnNames.get(i).name(), AvroRecordHelper.convertValueStringToAvroKeyType(getSchema(),
    +            columnNames.get(i).name(), keyMethodMap.get(i).get(tuple).toString()));
    +      } catch (AvroRuntimeException e) {
    +        LOG.error("Could not set Field [" + columnNames.get(i).name() + "] in the generic record");
    --- End diff --
    
    Please log exception.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56300505
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +
    +          }
    +
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +    }
    +
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   */
    +
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +
    +    }
    +
    +    return sb.substring(0, sb.length() - 1).toString();
    +
    +  }
    +
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +  };
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +
    +    columnFieldSetters = Lists.newArrayList();
    +
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +
    +    if (getFieldInfos().isEmpty() || getGenericRecordToPOJOFieldsMapping() == null) {
    +      setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +    } else {
    +      setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +    }
    +
    +    initColumnFieldSetters(getFieldInfos());
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    // TODO Auto-generated method stub
    +
    +  }
    +
    +  @Override
    +  public void beginWindow(long windowId)
    +  {
    +    // TODO Auto-generated method stub
    +
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    --- End diff --
    
    Do we need this getter setter provided the class is initialized from PortContext.TUPLE_CLASS


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56290586
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for Avro converted records
    + */
    +
    --- End diff --
    
    Please remove this space. The maven cannot find where to put @since tag then.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845120
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    --- End diff --
    
    new String(<getfield>).toString() shouldn't be needed? Isn't it? You could just do <getfield>


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287519
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    --- End diff --
    
    Can you generate setters and getters only once during setup or first time tuple is received, no need to generated them for each record.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287526
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +
    +          }
    +
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +    }
    +
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   */
    +
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +
    +    }
    +
    +    return sb.substring(0, sb.length() - 1).toString();
    --- End diff --
    
    toString() not needed


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56626092
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -114,6 +107,7 @@ private static Object convertAndResolveUnionToPrimitive(Schema schema, String ke
             }
             convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
           } catch (RuntimeException e) {
    +        LOG.error("Could not handle schema resolution" + e.getMessage());
    --- End diff --
    
    Actually failure for reading a single field inside a union should not cause a failure to read the rest of the values inside the union.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844759
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    private int recordCount... Same for other autometrics too..


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298711
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    --- End diff --
    
    Please have a valid javadoc.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297980
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    --- End diff --
    
    I see tuple being received by this Operator is a GenericRecord which is a avro type.
    Who is expected to create that? Upstream operator?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299041
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  /**
    +   * @param -
    +   *          className
    +   * @param -
    +   *          name of the field to create the getter for Returns a getter
    +   */
    +
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * @param -
    +   *          schema of the generic record Assumption is that the name of a
    +   *          field in POJO is the same as the name in Avro schema
    +   */
    +
    +  public void initializeColumnMap(Schema schema)
    +  {
    +    columnNames = schema.getFields();
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < columnNames.size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, columnNames.get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = true, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +
    +      GenericRecord record = null;
    +
    +      try {
    +        record = getGenericRecord(tuple);
    +      } catch (Exception e) {
    +        LOG.error("Exception in parsing record");
    +        errorCnt++;
    +      }
    +
    +      if (record != null) {
    +        output.emit(record);
    +        recordCnt++;
    +      } else {
    +        errorCnt++;
    +      }
    +
    +    }
    +
    +  };
    +
    +  /**
    +   * @param -Object
    --- End diff --
    
    Please have a complete javadoc.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57843812
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    --- End diff --
    
    Please do not hide the exception, atleast log the exception.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57766680
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,159 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + * @since 3.3.0
    --- End diff --
    
    Removed 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57846153
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class PojoToAvro extends BaseOperator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private transient String schemaString;
    +
    +  private transient Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  private void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  /**
    +   * Returns the schema string for Avro Generic Record
    +   * 
    +   * @return schemaString
    +   */
    +  public String getSchemaString()
    +  {
    +    return schemaString;
    +  }
    +
    +  /**
    +   * Sets the schema string
    +   */
    +  public void setSchemaString(String schemaString)
    +  {
    +    this.schemaString = schemaString;
    +  }
    +
    +  /**
    +   * Returns the schema object
    +   * 
    +   * @return schema
    +   */
    +  private Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  /**
    +   * Sets the shcema object
    +   */
    +  private void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  /**
    +   * Returns the list for field names from provided Avro schema
    +   * 
    +   * @return List of Fields
    +   */
    +  private List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  /**
    +   * Sets the list of column names representing the fields in Avro schema
    +   */
    +  private void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  /**
    +   * This method generates the getters for provided field of a given class
    +   * 
    +   * @return Getter
    +   */
    +  private Getter<?, ?> generateGettersForField(Class<?> cls, String inputFieldName)
    +      throws NoSuchFieldException, SecurityException
    +  {
    +    java.lang.reflect.Field f = cls.getDeclaredField(inputFieldName);
    +    Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +
    +    Getter<?, ?> classGetter = PojoUtils.createGetter(cls, inputFieldName, c);
    +
    +    return classGetter;
    +  }
    +
    +  /**
    +   * Initializes the list of columns in POJO based on the names from schema
    +   */
    +  private void initializeColumnMap(Schema schema)
    +  {
    +    setColumnNames(schema.getFields());
    +
    +    keyMethodMap = new ArrayList<Getter>();
    +    for (int i = 0; i < getColumnNames().size(); i++) {
    +      try {
    +        keyMethodMap.add(generateGettersForField(cls, getColumnNames().get(i).name()));
    +      } catch (NoSuchFieldException | SecurityException e) {
    +        throw new RuntimeException("Failed to initialize pojo class getters for field: ", e);
    +      }
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false, schemaRequired = true)
    +  public final transient DefaultInputPort<Object> data = new DefaultInputPort<Object>()
    +  {
    +
    +    @Override
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      try {
    +        parseSchema();
    +        initializeColumnMap(getSchema());
    +      } catch (IOException e) {
    +        LOG.error("Exception in parsing schema -" + e.getMessage());
    +      }
    +    }
    +
    +    @Override
    +    public void process(Object tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +
    +  };
    +
    +  /**
    +   * Converts incoming tuples into Generic records
    +   */
    +  protected void processTuple(Object tuple)
    +  {
    +    GenericRecord record = null;
    +
    +    try {
    +      record = getGenericRecord(tuple);
    +    } catch (Exception e) {
    +      LOG.error("Exception in parsing record");
    +      errorCnt++;
    +    }
    +
    +    if (record != null) {
    +      output.emit(record);
    +      recordCnt++;
    +    } else {
    +      errorCnt++;
    --- End diff --
    
    If conversion fails, shouldn't this be going to error output port?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57841961
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    +        throw new NullPointerException();
    +      }
    +    }
    +    return is;
    +  }
    +
    +  @Override
    +  protected GenericRecord readEntity() throws IOException
    +  {
    +    return readRecord();
    +  }
    +
    +  /**
    +   * Reads a GenericRecord from the given input stream<br>
    +   * Emits the FileName,Offset,Exception on the error port if its connected
    +   * 
    +   * @return GenericRecord
    +   */
    +  private GenericRecord readRecord() throws IOException
    +  {
    +    record = null;
    +
    +    try {
    +      if (avroDataStream != null && avroDataStream.hasNext()) {
    +        offset++;
    +
    +        record = avroDataStream.next();
    +        recordCnt++;
    +        return record;
    +      }
    +    } catch (AvroRuntimeException are) {
    +      LOG.error("Exception in parsing record for file - " + super.currentFile + " at offset - " + offset, are);
    +      if (errorRecordsPort.isConnected()) {
    +        errorRecordsPort.emit("FileName:" + super.currentFile + ", Offset:" + offset);
    +      }
    +      errorCnt++;
    +      throw new AvroRuntimeException(are);
    +    }
    +    return record;
    +  }
    +
    +  @Override
    +  protected void closeFile(InputStream is) throws IOException
    +  {
    +    String fileName = super.currentFile;
    +    super.closeFile(is);
    --- End diff --
    
    Shouldn't suport.closeFile be done later and avroDataStream.close() first? This is because avroStream is created from super.inputStream.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287547
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    --- End diff --
    
    extends from Parser abstract class.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299236
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroReaderWriterTest.java ---
    @@ -0,0 +1,219 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.ListIterator;
    +
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +
    +import junit.framework.Assert;
    +
    +@SuppressWarnings("deprecation")
    --- End diff --
    
    Please have warning suppression at specific places if that's needed and not put this as a general warning suppression at class level.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56294963
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    --- End diff --
    
    This operator is not working like a parser. The intent is to have a module for Avro of which this will be a part.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

Posted by devtagare <gi...@git.apache.org>.
GitHub user devtagare reopened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/211

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

    Please review changes for Avro to Pojo conversion & back.
    
    Thanks,
    Dev

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

    $ git pull https://github.com/devtagare/incubator-apex-malhar APEXMALHAR-2011-2012

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

    https://github.com/apache/incubator-apex-malhar/pull/211.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 #211
    
----
commit 12af6b62ada24417fddb78b533d71d3cf6dc58a1
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-15T14:39:06Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter

commit 6f9f301cad64577556991898966b9bd232d8e7db
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T13:29:43Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Review comments

commit 57600a4dfb727265644d093bd2cdeec46e917273
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-17T14:41:18Z

    APEXMALHAR-2011-2012 Avro to Pojo & Pojo to Avro converter - Removed unused spaces & variables

commit eff2510a77accea89c363de538d205267ba56ef5
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T00:09:44Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 4ea1f8421e5e49d268b3ad0523febd673eb65762
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:30:25Z

    APEXMALHAR-2011-2012 Avro File Input Operator

commit 9b785e1281dc9b645668d48959fee64cb4f871c0
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T20:37:46Z

    APEXMALHAR-2011-2012 Avro File Input Operator - Added documentation

commit 4b2b80e8a0cd2fa4f9a1ca3b768f858657f572e3
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T21:10:49Z

    APEXMALHAR-2011-2012 Avro File Input Operator - removed default constructor

commit 9a6bfa71fbc9d653a6d0012d6ac54b59dd11fae2
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-24T22:54:48Z

    Merge branch 'master' of https://github.com/apache/incubator-apex-malhar into APEXMALHAR-2011-2012

commit 163dfb58e8b02a76565f1f3ce509a6a28adcc85e
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-25T16:48:13Z

    Merge branch 'master' of https://github.com/apache/incubator-apex-malhar into APEXMALHAR-2011-2012

commit 4dd11a0a4f7e50c9036dc79dca05bb15748df24f
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-29T18:31:28Z

    APEXMALHAR-2011-2012 Avro File Input Operator - review comments

commit c03fb0694aafa70ff0221efbe7a96559bfc9be2b
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-29T18:33:09Z

    APEXMALHAR-2011-2012 Avro File Input Operator - review comments

commit c0224c1c0eff8bf2cedbc0411e05258653e8a1f1
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-29T18:37:12Z

    APEXMALHAR-2011-2012 Avro File Input Operator - review comments, checkstyle changes

commit 96f05fbd61ec011217f67c9d7777bc748f33ce68
Author: Devendra Tagare <de...@devendras-macbook-pro.local>
Date:   2016-03-29T18:39:18Z

    Merge branch 'master' of https://github.com/apache/incubator-apex-malhar into APEXMALHAR-2011-2012

----


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297800
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +        if (obj != null) {
    +          output.emit(obj);
    +          recordCnt++;
    +        }
    +
    +      } catch (InstantiationException | IllegalAccessException e) {
    +        LOG.error("Could not initialize object of class - " + getClass().getName());
    +        errorCnt++;
    +      }
    +    }
    +
    +  };
    +
    +  @SuppressWarnings("unchecked")
    +  public Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +
    +        afi = columnFieldSetters.get(i);
    +
    +        SupportType st = afi.fieldInfo.getType();
    +
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +
    +          switch (st) {
    +
    +            case BOOLEAN:
    +
    +              afi.setterOrGetter = PojoUtils.createSetterBoolean(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              afi.setterOrGetter = PojoUtils.createSetterDouble(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              afi.setterOrGetter = PojoUtils.createSetterFloat(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              afi.setterOrGetter = PojoUtils.createSetterInt(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(),
    +                  afi.fieldInfo.getType().getJavaType());
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              afi.setterOrGetter = PojoUtils.createSetterLong(getCls(), afi.fieldInfo.getPojoFieldExpression());
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              afi.setterOrGetter = PojoUtils.createSetter(getCls(), afi.fieldInfo.getPojoFieldExpression(), Byte.class);
    --- End diff --
    
    What is this default case meant for?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297209
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public byte[] serialize(Object obj) throws IOException
    +  {
    +    try (ByteArrayOutputStream b = new ByteArrayOutputStream()) {
    +      try (ObjectOutputStream o = new ObjectOutputStream(b)) {
    +        o.writeObject(obj);
    +      }
    +      return b.toByteArray();
    +    }
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +
    +      try {
    +        Object obj = getPOJOFromGenericRecord(tuple, getCls());
    --- End diff --
    
    Please move this code from process method at the class level and then call that method from here.
    This is so that if one wants to extend this, he/she can override the process functionality too.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56297505
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private transient List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public void parseSchema() throws IOException
    --- End diff --
    
    Any reason for this method to be public? If its not meant to be accessed outside of the class, please make this private or protected appropriately.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298519
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroRecordHelper.java ---
    @@ -0,0 +1,124 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.text.ParseException;
    +import java.util.List;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Type;
    +
    +/**
    + * This is an utility class for Avro converted records
    + */
    +
    +public class AvroRecordHelper
    +{
    +
    +  /**
    +   * Convert a passed String value to the given type for the key as per Schema
    +   * 
    +   * @param schema
    +   * @param key
    +   * @param value
    +   * @return
    +   */
    +
    +  public static Object convertValueStringToAvroKeyType(Schema schema, String key, String value) throws ParseException
    +  {
    +    Type type = null;
    +
    +    if (schema.getField(key) != null) {
    +      type = schema.getField(key).schema().getType();
    +    } else {
    +      return value;
    +    }
    +
    +    Object convertedValue = null;
    +
    +    if (type == Type.UNION) {
    +      convertedValue = convertAndResolveUnionToPrimitive(schema, key, value);
    +    } else {
    +      convertedValue = convertValueToAvroPrimitive(type, key, value);
    +    }
    +
    +    return convertedValue;
    +
    +  }
    +
    +  private static Object convertValueToAvroPrimitive(Type type, String key, String value) throws ParseException
    +  {
    +    Object newValue = value;
    +    switch (type) {
    +      case BOOLEAN:
    +        newValue = Boolean.parseBoolean(value);
    +        break;
    +      case DOUBLE:
    +        newValue = Double.parseDouble(value);
    +        break;
    +      case FLOAT:
    +        newValue = Float.parseFloat(value);
    +        break;
    +      case INT:
    +        newValue = Integer.parseInt(value);
    +        break;
    +      case LONG:
    +        newValue = Long.parseLong(value);
    +        break;
    +      case BYTES:
    +        newValue = value.getBytes();
    +        break;
    +      case STRING:
    +        newValue = value;
    +        break;
    +      case NULL:
    +        newValue = null;
    +        break;
    +      default:
    +        newValue = value;
    +    }
    +    return newValue;
    +  }
    +
    +  private static Object convertAndResolveUnionToPrimitive(Schema schema, String key, String value) throws ParseException
    +  {
    +    Schema unionSchema = schema.getField(key).schema();
    +    List<Schema> types = unionSchema.getTypes();
    +    Object convertedValue = null;
    +    for (int i = 0; i < types.size(); i++) {
    +      try {
    +        if (types.get(i).getType() == Type.NULL) {
    +          if (value == null || value.equals("null")) {
    +            convertedValue = null;
    +            break;
    +          } else {
    +            continue;
    +          }
    +        }
    +        convertedValue = convertValueToAvroPrimitive(types.get(i).getType(), key, value);
    +      } catch (RuntimeException e) {
    +        continue;
    --- End diff --
    
    Shouldn't we at least log the exception?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57936223
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Had a chat with @chandnisingh. Marking them private & marking @VisibleForTesting


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57845826
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    --- End diff --
    
    Please remove @since.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844505
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    +      errorCnt++;
    +      newObj = null;
    +    }
    +    return newObj;
    +  }
    +
    +  /**
    +   * Use reflection to generate field info values if the user has not provided
    +   * the inputs mapping
    +   * 
    +   * @return String representing the POJO field to Avro field mapping
    +   */
    +  public String generateFieldInfoInputs(Class<?> cls)
    +  {
    +    java.lang.reflect.Field[] fields = cls.getDeclaredFields();
    +    StringBuilder sb = new StringBuilder();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      java.lang.reflect.Field f = fields[i];
    +      Class<?> c = ClassUtils.primitiveToWrapper(f.getType());
    +      sb.append(f.getName() + FIELD_SEPARATOR + f.getName() + FIELD_SEPARATOR + c.getSimpleName().toUpperCase()
    +          + RECORD_SEPARATOR);
    +    }
    +    return sb.substring(0, sb.length() - 1);
    +  }
    +
    +  /**
    +   * Creates a map representing fieldName in POJO:field in Generic Record:Data
    +   * type
    +   * 
    +   * @return List of FieldInfo
    +   */
    +  public List<FieldInfo> createFieldInfoMap(String str)
    +  {
    +    fieldInfos = new ArrayList<FieldInfo>();
    +    StringTokenizer strtok = new StringTokenizer(str, RECORD_SEPARATOR);
    +
    +    while (strtok.hasMoreTokens()) {
    +      String[] token = strtok.nextToken().split(FIELD_SEPARATOR);
    +      fieldInfos.add(new FieldInfo(token[0], token[1], SupportType.valueOf(token[2])));
    +    }
    +    return fieldInfos;
    +  }
    +
    +  @OutputPortFieldAnnotation(schemaRequired = true)
    +  public final transient DefaultOutputPort<Object> output = new DefaultOutputPort<Object>()
    +  {
    +    public void setup(PortContext context)
    +    {
    +      cls = context.getValue(Context.PortContext.TUPLE_CLASS);
    +
    +      columnFieldSetters = Lists.newArrayList();
    +
    +      if (getGenericRecordToPOJOFieldsMapping() == null) {
    +        setFieldInfos(createFieldInfoMap(generateFieldInfoInputs(getCls())));
    +      } else {
    +        setFieldInfos(createFieldInfoMap(getGenericRecordToPOJOFieldsMapping()));
    +      }
    +
    +      initColumnFieldSetters(getFieldInfos());
    +      initializeActiveFieldSetters();
    +    }
    +  };
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    errorCnt = 0;
    +    fieldErrorCnt = 0;
    +    recordCnt = 0;
    +
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  /**
    +   * Class that maps fieldInfo to its getters or setters
    +   */
    +  protected static class ActiveFieldInfo
    +  {
    +    final FieldInfo fieldInfo;
    +    Object setterOrGetter;
    +
    +    ActiveFieldInfo(FieldInfo fieldInfo)
    +    {
    +      this.fieldInfo = fieldInfo;
    +    }
    +  }
    +
    +  /**
    +   * A list of {@link FieldInfo}s where each item maps a column name to a pojo
    +   * field name.
    +   */
    +  private List<FieldInfo> getFieldInfos()
    +  {
    +    return fieldInfos;
    +  }
    +
    +  /**
    +   * Add the Active Fields to the columnFieldSetters {@link ActiveFieldInfo}s
    +   */
    +  public void initColumnFieldSetters(List<FieldInfo> fieldInfos)
    +  {
    +    for (FieldInfo fi : fieldInfos) {
    +      if (columnFieldSetters == null) {
    +        columnFieldSetters = Lists.newArrayList();
    +      }
    +      columnFieldSetters.add(new AvroToPojo.ActiveFieldInfo(fi));
    +    }
    +  }
    +
    +  /**
    +   * Sets the {@link FieldInfo}s. A {@link FieldInfo} maps a store column to a
    +   * pojo field name.<br/>
    +   * The value from fieldInfo.column is assigned to
    +   * fieldInfo.pojoFieldExpression.
    +   *
    +   * @description $[].columnName name of the Output Field in POJO
    +   * @description $[].pojoFieldExpression expression to get the respective field
    +   *              from generic record
    +   * @useSchema $[].pojoFieldExpression outputPort.fields[].name
    +   */
    +  public void setFieldInfos(List<FieldInfo> fieldInfos)
    +  {
    +    this.fieldInfos = fieldInfos;
    +  }
    +
    +  /**
    +   * Initialize the setters for generating the POJO
    +   */
    +  private void initializeActiveFieldSetters()
    +  {
    +    for (int i = 0; i < columnFieldSetters.size(); i++) {
    +      ActiveFieldInfo activeFieldInfo = columnFieldSetters.get(i);
    +
    +      SupportType st = activeFieldInfo.fieldInfo.getType();
    +
    +      switch (st) {
    +
    +        case BOOLEAN:
    +
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetterBoolean(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression());
    +          break;
    +
    +        case DOUBLE:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetterDouble(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression());
    +          break;
    +
    +        case FLOAT:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetterFloat(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression());
    +          break;
    +
    +        case INTEGER:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetterInt(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression());
    +          break;
    +
    +        case STRING:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetter(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression(), activeFieldInfo.fieldInfo.getType().getJavaType());
    +          break;
    +
    +        case LONG:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetterLong(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression());
    +          break;
    +
    +        default:
    +          activeFieldInfo.setterOrGetter = PojoUtils.createSetter(getCls(),
    +              activeFieldInfo.fieldInfo.getPojoFieldExpression(), Byte.class);
    +          break;
    +      }
    +
    +      columnFieldSetters.get(i).setterOrGetter = activeFieldInfo.setterOrGetter;
    +    }
    +  }
    +
    +  /**
    +   * Serializes a given object, Added as a plug to serialize non-primitive types
    +   * 
    +   * @return byte[]
    +   */
    +  private byte[] serialize(Object obj) throws IOException
    --- End diff --
    
    Can you please add class level javadoc comment, if this comes in action and how would the POJO get populated with 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.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57841247
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  private transient DataFileStream<GenericRecord> avroDataStream;
    +  private transient GenericRecord record = null;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  public final transient DefaultOutputPort<String> completedFilesPort = new DefaultOutputPort<String>();
    +
    +  public final transient DefaultOutputPort<String> errorRecordsPort = new DefaultOutputPort<String>();
    +
    +  /**
    +   * Returns a input stream given a file path
    +   * 
    +   * @param path
    +   * @return InputStream
    +   * @throws IOException
    +   */
    +  @Override
    +  protected InputStream openFile(Path path) throws IOException
    +  {
    +    InputStream is = super.openFile(path);
    +    if (is != null) {
    +      try {
    +        DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
    +        avroDataStream = new DataFileStream<GenericRecord>(is, datumReader);
    +        datumReader.setSchema(avroDataStream.getSchema());
    +      } catch (NullPointerException npe) {
    +        LOG.error("Schemaless file", npe);
    --- End diff --
    
    If you anyway to throw exception here.. Then why catch it? Is there specific advantage that you see with logging this instead of throwing to parent?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57842065
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroFileInputOperator.java ---
    @@ -0,0 +1,164 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.file.DataFileStream;
    +import org.apache.avro.generic.GenericDatumReader;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumReader;
    +import org.apache.hadoop.classification.InterfaceStability;
    +import org.apache.hadoop.fs.Path;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.lib.io.fs.AbstractFileInputOperator;
    +
    +/**
    + * <p>
    + * Avro File Input Operator
    + * </p>
    + * A specific implementation of the AbstractFileInputOperator to read Avro
    + * container files.<br>
    + * No need to provide schema,its inferred from the file<br>
    + * Users can add the {@link IdempotentStorageManager.FSIdempotentStorageManager}
    + * to ensure exactly once semantics with a HDFS backed WAL.
    + * 
    + * @displayName AvroFileInputOperator
    + * @category Input
    + * @tags fs, file,avro, input operator
    + */
    +@InterfaceStability.Evolving
    +public class AvroFileInputOperator extends AbstractFileInputOperator<GenericRecord>
    +{
    +
    +  private transient long offset = 0L;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Can you give a complete name recordCount as this name will appear as metric. Samefor errorCnt.. Please make it recordCount and errorCount.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57843723
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    --- End diff --
    
    setting input port to optional=false is not required. Its false by default.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56294938
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    --- End diff --
    
    This operator is not working like a parser. The intent is to have a module for Avro of which this will be a part.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57843644
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    --- End diff --
    
    Same for fieldErrorCount


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56299411
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/Order.java ---
    @@ -0,0 +1,49 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.Serializable;
    +
    +public class Order implements Serializable
    +{
    +
    +  /**
    --- End diff --
    
    Please remove unnecessary 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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r58004569
  
    --- Diff: contrib/src/test/java/com/datatorrent/contrib/avro/AvroToPojoTest.java ---
    @@ -0,0 +1,294 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.util.List;
    +import java.util.ListIterator;
    +
    +import org.junit.Assert;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.TestWatcher;
    +import org.junit.runner.Description;
    +import org.python.google.common.collect.Lists;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +
    +import com.datatorrent.api.Attribute;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.helper.TestPortContext;
    +import com.datatorrent.lib.testbench.CollectorTestSink;
    +
    +public class AvroToPojoTest
    --- End diff --
    
    Added in the AvroFileInputOperator to test AvroFileInputOperator + AvroToPojo


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56287505
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,428 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class AvroToPojo implements Operator
    --- End diff --
    
    can you extend it from the Parser abstract class, and override convert method which will convert to GenericObject to Pojo.


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r56298639
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/PojoToAvro.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.Schema;
    +import org.apache.avro.Schema.Field;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.lib.util.PojoUtils;
    +import com.datatorrent.lib.util.PojoUtils.Getter;
    +
    +/**
    + * <p>
    + * PojoToAvro
    + * </p>
    + * A generic implementation for POJO to Avro conversion. A POJO is converted to
    + * a GenericRecord based on the schema provided. As of now only primitive types
    + * are supported.
    + *
    + * @displayName Pojo To Avro
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +
    +@InterfaceStability.Evolving
    +public class PojoToAvro implements Operator
    +{
    +
    +  private List<Field> columnNames;
    +
    +  private Class<?> cls;
    +
    +  private List<Getter> keyMethodMap;
    +
    +  private String schemaFile;
    +
    +  private Schema schema;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  public final transient DefaultOutputPort<GenericRecord> output = new DefaultOutputPort<GenericRecord>();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    try {
    +      parseSchema();
    +    } catch (IOException e) {
    +      LOG.error("Exception in parsing schema");
    +    }
    +    initializeColumnMap(getSchema());
    +  }
    +
    +  public void parseSchema() throws IOException
    +  {
    +    setSchema(new Schema.Parser().parse(getSchemaString()));
    +  }
    +
    +  public String getSchemaString()
    +  {
    +    return schemaFile;
    +  }
    +
    +  public void setSchemaString(String schemaFile)
    +  {
    +    this.schemaFile = schemaFile;
    +  }
    +
    +  public Schema getSchema()
    +  {
    +    return schema;
    +  }
    +
    +  public void setSchema(Schema schema)
    +  {
    +    this.schema = schema;
    +  }
    +
    +  public List<Field> getColumnNames()
    +  {
    +    return columnNames;
    +  }
    +
    +  public void setColumnNames(List<Field> columnNames)
    +  {
    +    this.columnNames = columnNames;
    +  }
    +
    +  public Class<?> getCls()
    +  {
    +    return cls;
    +  }
    +
    +  public void setCls(Class<?> cls)
    +  {
    +    this.cls = cls;
    +  }
    +
    +  public List<Getter> getKeyMethodMap()
    +  {
    +    return keyMethodMap;
    +  }
    +
    +  public void setKeyMethodMap(List<Getter> keyMethodMap)
    +  {
    +    this.keyMethodMap = keyMethodMap;
    +  }
    +
    +  /**
    +   * Adding this as a plug for being able to serialize non primitive types
    +   * 
    +   * @param -
    +   *          object to serialize Returns a byte array
    +   */
    +
    +  public byte[] serialize(Object obj) throws IOException
    --- End diff --
    
    why public?


---
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] incubator-apex-malhar pull request: APEXMALHAR-2011-2012 Avro to P...

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

    https://github.com/apache/incubator-apex-malhar/pull/211#discussion_r57844582
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/avro/AvroToPojo.java ---
    @@ -0,0 +1,415 @@
    +/**
    + * 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 com.datatorrent.contrib.avro;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectOutputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.StringTokenizer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.avro.AvroRuntimeException;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.commons.lang3.ClassUtils;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.google.common.collect.Lists;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.PortContext;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.DefaultOutputPort;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.lib.util.FieldInfo;
    +import com.datatorrent.lib.util.FieldInfo.SupportType;
    +import com.datatorrent.lib.util.PojoUtils;
    +
    +/**
    + * <p>
    + * AvroToPojo
    + * </p>
    + * A generic implementation for conversion from Avro to POJO. The POJO class
    + * name & field mapping should be provided by the user. If this mapping is not
    + * provided then reflection is used to determine this mapping. As of now only
    + * primitive types are supported.
    + *
    + * @displayName Avro To Pojo
    + * @category Converter
    + * @tags avro
    + * @since 3.3.0
    + */
    +@InterfaceStability.Evolving
    +public class AvroToPojo extends BaseOperator
    +{
    +
    +  private Class<?> cls;
    +
    +  private static final String FIELD_SEPARATOR = ":";
    +  private static final String RECORD_SEPARATOR = ",";
    +
    +  private String genericRecordToPOJOFieldsMapping = null;
    +
    +  private List<FieldInfo> fieldInfos;
    +
    +  private List<ActiveFieldInfo> columnFieldSetters = null;
    +
    +  @AutoMetric
    +  int recordCnt = 0;
    +
    +  @AutoMetric
    +  int errorCnt = 0;
    +
    +  @AutoMetric
    +  int fieldErrorCnt = 0;
    +
    +  /**
    +   * Retruns a string representing mapping from generic record to POJO fields
    +   */
    +  public String getGenericRecordToPOJOFieldsMapping()
    +  {
    +    return genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  /**
    +   * Comma separated list mapping a field in Avro schema to POJO field eg :
    +   * orderId:orderId:INTEGER
    +   */
    +  public void setGenericRecordToPOJOFieldsMapping(String genericRecordToPOJOFieldsMapping)
    +  {
    +    this.genericRecordToPOJOFieldsMapping = genericRecordToPOJOFieldsMapping;
    +  }
    +
    +  @InputPortFieldAnnotation(optional = false)
    +  public final transient DefaultInputPort<GenericRecord> data = new DefaultInputPort<GenericRecord>()
    +  {
    +    @Override
    +    public void process(GenericRecord tuple)
    +    {
    +      processTuple(tuple);
    +    }
    +  };
    +
    +  /**
    +   * Converts given Generic Record and to a POJO and emits it
    +   */
    +  protected void processTuple(GenericRecord tuple)
    +  {
    +    try {
    +      Object obj = getPOJOFromGenericRecord(tuple, getCls());
    +
    +      if (obj != null) {
    +        output.emit(obj);
    +        recordCnt++;
    +      }
    +
    +    } catch (InstantiationException | IllegalAccessException e) {
    +      LOG.error("Could not initialize object of class - " + getClass().getName());
    +      errorCnt++;
    +    }
    +  }
    +
    +  /**
    +   * Returns a POJO from a Generic Record
    +   * 
    +   * @return Object
    +   */
    +  @SuppressWarnings("unchecked")
    +  private Object getPOJOFromGenericRecord(GenericRecord tuple, Class<?> cls)
    +      throws InstantiationException, IllegalAccessException
    +  {
    +    Object newObj = getCls().newInstance();
    +
    +    try {
    +      for (int i = 0; i < columnFieldSetters.size(); i++) {
    +
    +        AvroToPojo.ActiveFieldInfo afi = columnFieldSetters.get(i);
    +        afi = columnFieldSetters.get(i);
    +        SupportType st = afi.fieldInfo.getType();
    +        Object val = null;
    +
    +        try {
    +          val = tuple.get(afi.fieldInfo.getColumnName());
    +        } catch (Exception e) {
    +          LOG.error("Could not find field -" + afi.fieldInfo.getColumnName() + "- in the generic record");
    +          val = null;
    +          fieldErrorCnt++;
    +        }
    +
    +        if (val == null) {
    +          continue;
    +        }
    +
    +        try {
    +          switch (st) {
    +            case BOOLEAN:
    +              ((PojoUtils.SetterBoolean<Object>)afi.setterOrGetter).set(newObj,
    +                  (boolean)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case DOUBLE:
    +              ((PojoUtils.SetterDouble<Object>)afi.setterOrGetter).set(newObj,
    +                  (double)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case FLOAT:
    +              ((PojoUtils.SetterFloat<Object>)afi.setterOrGetter).set(newObj,
    +                  (float)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case INTEGER:
    +              ((PojoUtils.SetterInt<Object>)afi.setterOrGetter).set(newObj,
    +                  (int)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            case STRING:
    +              ((PojoUtils.Setter<Object, String>)afi.setterOrGetter).set(newObj,
    +                  new String(tuple.get(afi.fieldInfo.getColumnName()).toString()));
    +              break;
    +
    +            case LONG:
    +              ((PojoUtils.SetterLong<Object>)afi.setterOrGetter).set(newObj,
    +                  (long)tuple.get(afi.fieldInfo.getColumnName()));
    +              break;
    +
    +            default:
    +              ((PojoUtils.Setter<Object, byte[]>)afi.setterOrGetter).set(newObj,
    +                  serialize(tuple.get(afi.fieldInfo.getColumnName())));
    +              break;
    +          }
    +        } catch (AvroRuntimeException | IOException e) {
    +          LOG.error("Exception in setting value" + e.getMessage());
    +          fieldErrorCnt++;
    +        }
    +
    +      }
    +    } catch (Exception ex) {
    +      LOG.error("Generic Exception in setting value" + ex.getMessage());
    --- End diff --
    
    You could just add ex to LOG.error and it'll call its toString method. getMessage might not print callstack.


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