You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by shubham-pathak22 <gi...@git.apache.org> on 2015/09/11 13:01:54 UTC

[GitHub] incubator-apex-malhar pull request: MLHR-1838 added parser and for...

GitHub user shubham-pathak22 opened a pull request:

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

    MLHR-1838 added parser and formatter

    Please review.
    Added parsers and test scripts for
    XML,CSV,JSON->POJO
    Added formatters and test scripts for 
    POJO->XML,CSV,JSON

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

    $ git pull https://github.com/shubham-pathak22/incubator-apex-malhar MLHR-1838

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

    https://github.com/apache/incubator-apex-malhar/pull/22.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 #22
    
----
commit 8e2774e5b68d638724eddde0e792d97dd856050d
Author: shubham <sh...@github.com>
Date:   2015-09-11T10:56:03Z

    MLHR-1838 added parser and formatter

----


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41665402
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/XmlFormatter.java ---
    @@ -0,0 +1,172 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.Writer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +
    +import com.thoughtworks.xstream.XStream;
    +import com.thoughtworks.xstream.XStreamException;
    +import com.thoughtworks.xstream.converters.basic.DateConverter;
    +import com.thoughtworks.xstream.io.HierarchicalStreamWriter;
    +import com.thoughtworks.xstream.io.xml.CompactWriter;
    +import com.thoughtworks.xstream.io.xml.XppDriver;
    +
    +/**
    + * @displayName XmlParser
    + * @category Formatter
    + * @tags xml pojo formatter
    + */
    +@InterfaceStability.Evolving
    +public class XmlFormatter extends Formatter<String>
    +{
    +
    +  private transient XStream xstream;
    +
    +  protected String alias;
    +  protected String dateFormat;
    +  protected boolean prettyPrint;
    +
    +  public XmlFormatter()
    +  {
    +    alias = null;
    +    dateFormat = null;
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    if (prettyPrint) {
    +      xstream = new XStream();
    +    } else {
    +      xstream = new XStream(new XppDriver()
    +      {
    +        @Override
    +        public HierarchicalStreamWriter createWriter(Writer out)
    +        {
    +          return new CompactWriter(out, getNameCoder());
    +        }
    +      });
    +    }
    +    if (alias != null) {
    +      try {
    +        xstream.alias(alias, clazz);
    +      } catch (Throwable e) {
    +        throw new RuntimeException("Unable find provided class");
    +      }
    +    }
    +    if (dateFormat != null) {
    +      xstream.registerConverter(new DateConverter(dateFormat, new String[] {}));
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      return xstream.toXML(tuple);
    +    } catch (XStreamException e) {
    +      logger.error("Error while converting tuple {} ", tuple + " " + e.getMessage());
    --- End diff --
    
    same as above.


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599903
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.exception.SuperCsvException;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +@InterfaceStability.Evolving
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] fieldInfoTuple = fieldInfo.split(",");
    +    for ( int i=0 ; i < fieldInfoTuple.length ; i++ ) {
    +      String[] fieldTuple = fieldInfoTuple[i].split(":");
    +      Field field = new Field();
    +      field.setName(fieldTuple[0]);
    +      String[] typeFormat = fieldTuple[1].split("\\|");
    +      field.setType(typeFormat[0].toUpperCase());
    +      if ( typeFormat.length > 1 ) {
    +        field.setFormat(typeFormat[1]);
    +      }
    +      getFields().add(field);
    +    }
    +    preference = new CsvPreference.Builder('"', fieldDelimiter, lineDelimiter).build();
    +    int countKeyValue = getFields().size();
    +    nameMapping = new String[countKeyValue];
    +    processors = new CellProcessor[countKeyValue];
    +    initialise(nameMapping, processors);
    +   
    +  }
    +
    +  private void initialise(String[] nameMapping, CellProcessor[] processors)
    +  {
    +    for (int i = 0; i < getFields().size(); i++) {
    +      FIELD_TYPE type = getFields().get(i).type;
    +      nameMapping[i] = getFields().get(i).name;
    +       if (type == FIELD_TYPE.DATE) {
    +        String dateFormat = getFields().get(i).format;
    +        processors[i] = new Optional(new FmtDate(dateFormat == null ? "dd/MM/yyyy" : dateFormat));
    +      }else{
    +        processors[i] = new Optional();
    +      }
    +    }
    +
    +  }
    --- End diff --
    
    newline after function.


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599898
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.exception.SuperCsvException;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +@InterfaceStability.Evolving
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] fieldInfoTuple = fieldInfo.split(",");
    +    for ( int i=0 ; i < fieldInfoTuple.length ; i++ ) {
    +      String[] fieldTuple = fieldInfoTuple[i].split(":");
    +      Field field = new Field();
    +      field.setName(fieldTuple[0]);
    +      String[] typeFormat = fieldTuple[1].split("\\|");
    +      field.setType(typeFormat[0].toUpperCase());
    +      if ( typeFormat.length > 1 ) {
    --- End diff --
    
    inconsistent formatting space after ( in if, please format all files using formatter 
    https://github.com/chandnisingh/incubator-apex-core/tree/APEX-151/misc/ide-templates


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41665388
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/JsonFormatter.java ---
    @@ -0,0 +1,109 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.JsonGenerationException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectWriter;
    +import org.codehaus.jackson.map.SerializationConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to JSON string <br>
    + * <b>Properties</b> <br>
    + * <b>dateFormat</b>: date format e.g dd/MM/yyyy
    + * 
    + * @displayName JsonFormatter
    + * @category Formatter
    + * @tags pojo json formatter
    + */
    +@InterfaceStability.Evolving
    +public class JsonFormatter extends Formatter<String>
    +{
    +  private transient ObjectWriter writer;
    +  protected String dateFormat;
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      if (dateFormat != null) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      writer = mapper.writerWithType(clazz);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, true);
    +    } catch (Throwable e) {
    +      throw new RuntimeException("Unable find provided class");
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      return writer.writeValueAsString(tuple);
    +    } catch (JsonGenerationException | JsonMappingException e) {
    +      logger.error("Error while converting tuple {} ", tuple + " " + e.getMessage());
    --- End diff --
    
    no need to use + operator 
    (Error while converting tuple {} {}", tuple, e.getMessage())
    Also change a level to debug, as error tuples are emitted on a port, user can check the error tuple if required.


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

[GitHub] incubator-apex-malhar pull request: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41478262
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,268 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] field_info = fieldInfo.split(",");
    +    for ( int i=0 ; i < field_info.length ; i++ ) {
    +      String[] field_tuple = field_info[i].split(":");
    +      Field field = new Field();
    +      field.setName(field_tuple[0]);
    +      String[] type_format = field_tuple[1].split("\\|");
    --- End diff --
    
    Could you explain what you mean by Inconsistent variable naming ? Is it to do with using camel casing instead of underscore OR the name "type_format" must be renamed to more suitable name such as "field_type_format" ?


---
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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41477040
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/JsonFormatter.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectWriter;
    +import org.codehaus.jackson.map.SerializationConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to JSON string
    + * <br><b>Properties</b>
    + *  <br><b>dateFormat</b>: date format
    + *  e.g dd/MM/yyyy
    + * @displayName JsonFormatter
    + * @category Formatter 
    + * @tags  pojo json formatter
    + */
    +public class JsonFormatter extends Formatter<String>
    +{
    +
    +  private transient ObjectWriter writer;
    +  protected String dateFormat;
    +  
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      if (dateFormat != null ) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      writer = mapper.writerWithType(clazz);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, true);
    +    } 
    +    catch (Throwable e) {
    +      throw new RuntimeException("Unable find provided class");
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    // TODO Auto-generated method stub
    +    
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      return writer.writeValueAsString(tuple);
    +    }
    +    catch (Exception ex) {
    +      logger.error("error while deserializing tuple",ex.getMessage());
    --- End diff --
    
    Propagate 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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41477456
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,268 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] field_info = fieldInfo.split(",");
    +    for ( int i=0 ; i < field_info.length ; i++ ) {
    +      String[] field_tuple = field_info[i].split(":");
    +      Field field = new Field();
    +      field.setName(field_tuple[0]);
    +      String[] type_format = field_tuple[1].split("\\|");
    --- End diff --
    
    Inconsistent variable naming.


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599905
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.exception.SuperCsvException;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +@InterfaceStability.Evolving
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] fieldInfoTuple = fieldInfo.split(",");
    +    for ( int i=0 ; i < fieldInfoTuple.length ; i++ ) {
    +      String[] fieldTuple = fieldInfoTuple[i].split(":");
    +      Field field = new Field();
    +      field.setName(fieldTuple[0]);
    +      String[] typeFormat = fieldTuple[1].split("\\|");
    +      field.setType(typeFormat[0].toUpperCase());
    +      if ( typeFormat.length > 1 ) {
    +        field.setFormat(typeFormat[1]);
    +      }
    +      getFields().add(field);
    +    }
    +    preference = new CsvPreference.Builder('"', fieldDelimiter, lineDelimiter).build();
    +    int countKeyValue = getFields().size();
    +    nameMapping = new String[countKeyValue];
    +    processors = new CellProcessor[countKeyValue];
    +    initialise(nameMapping, processors);
    +   
    +  }
    +
    +  private void initialise(String[] nameMapping, CellProcessor[] processors)
    +  {
    +    for (int i = 0; i < getFields().size(); i++) {
    +      FIELD_TYPE type = getFields().get(i).type;
    +      nameMapping[i] = getFields().get(i).name;
    +       if (type == FIELD_TYPE.DATE) {
    +        String dateFormat = getFields().get(i).format;
    +        processors[i] = new Optional(new FmtDate(dateFormat == null ? "dd/MM/yyyy" : dateFormat));
    +      }else{
    +        processors[i] = new Optional();
    +      }
    +    }
    +
    +  }
    +  @Override
    +  public void activate(Context context)
    +  {
    +    
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      StringWriter stringWriter = new StringWriter();
    +      ICsvBeanWriter beanWriter = new CsvBeanWriter(stringWriter, preference);
    +      beanWriter.write(tuple, nameMapping,processors);
    +      beanWriter.flush();
    +      beanWriter.close();
    +      return stringWriter.toString();
    +    }
    +    catch (SuperCsvException e) {
    +      logger.error("Error while converting tuple {} ", tuple + " " + e.getMessage());
    +    }catch (IOException e) {
    --- End diff --
    
    space in between { and 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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41476959
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,268 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] field_info = fieldInfo.split(",");
    +    for ( int i=0 ; i < field_info.length ; i++ ) {
    +      String[] field_tuple = field_info[i].split(":");
    +      Field field = new Field();
    +      field.setName(field_tuple[0]);
    +      String[] type_format = field_tuple[1].split("\\|");
    +      field.setType(type_format[0].toUpperCase());
    +      if ( type_format.length > 1 ) {
    +        field.setFormat(type_format[1]);
    +      }
    +      getFields().add(field);
    +    }
    +    preference = new CsvPreference.Builder('"', fieldDelimiter, lineDelimiter).build();
    +    int countKeyValue = getFields().size();
    +    nameMapping = new String[countKeyValue];
    +    processors = new CellProcessor[countKeyValue];
    +    initialise(nameMapping, processors);
    +   
    +  }
    +
    +  private void initialise(String[] nameMapping, CellProcessor[] processors)
    +  {
    +    for (int i = 0; i < getFields().size(); i++) {
    +      FIELD_TYPE type = getFields().get(i).type;
    +      nameMapping[i] = getFields().get(i).name;
    +       if (type == FIELD_TYPE.DATE) {
    +        String dateFormat = getFields().get(i).format;
    +        processors[i] = new Optional(new FmtDate(dateFormat == null ? "dd/MM/yyyy" : dateFormat));
    +      }else{
    +        processors[i] = new Optional();
    +      }
    +    }
    +
    +  }
    +  @Override
    +  public void activate(Context context)
    +  {
    +    // TODO Auto-generated method stub
    +    
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    // TODO Auto-generated method stub
    +    
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      StringWriter stringWriter = new StringWriter();
    +      ICsvBeanWriter beanWriter = new CsvBeanWriter(stringWriter, preference);
    +      beanWriter.write(tuple, nameMapping,processors);
    +      beanWriter.flush();
    +      beanWriter.close();
    +      return stringWriter.toString();
    +    }
    +    catch (IOException e) {
    +      logger.error("Error while processing {} ", e.getMessage());
    --- End diff --
    
    Propagate 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: MLHR-1838 added parser and for...

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/22#discussion_r41599913
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/JsonFormatter.java ---
    @@ -0,0 +1,108 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.JsonGenerationException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectWriter;
    +import org.codehaus.jackson.map.SerializationConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to JSON string
    + * <br><b>Properties</b>
    + *  <br><b>dateFormat</b>: date format
    + *  e.g dd/MM/yyyy
    + * @displayName JsonFormatter
    + * @category Formatter 
    + * @tags  pojo json formatter
    + */
    +@InterfaceStability.Evolving
    +public class JsonFormatter extends Formatter<String>
    +{
    +  private transient ObjectWriter writer;
    +  protected String dateFormat;
    +  
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      if (dateFormat != null ) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      writer = mapper.writerWithType(clazz);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, true);
    +    } 
    --- End diff --
    
    use consistent formatting for  try catch, some place catch is at next line after } and in some places it is on the same line.


---
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: MLHR-1838 added parser and for...

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

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


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599910
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/Formatter.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.schema.formatter;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +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.Operator.ActivationListener;
    +import com.datatorrent.api.annotation.InputPortFieldAnnotation;
    +import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
    +import com.datatorrent.common.util.BaseOperator;
    +import com.datatorrent.contrib.converter.Converter;
    +
    +/**
    + * Abstract class that implements Converter interface.
    + * This is a schema enabled Formatter
    + * <br>
    + * Sub classes need to implement the convert method
    + * <br>
    + * <b>Port Interface</b><br>
    + * <b>in</b>: expects &lt;Object&gt; this is a schema enabled port<br>
    + * <b>out</b>: emits &lt;OUTPUT&gt; <br>
    + * <b>err</b>: emits &lt;Object&gt; error port that emits input tuple that could not be converted<br>
    + * <br>
    + * @displayName Parser
    + * @tags parser converter
    + * @param <INPUT>
    + */
    +@InterfaceStability.Evolving
    +public abstract class Formatter <OUTPUT> extends BaseOperator implements Converter<Object,OUTPUT>,ActivationListener<Context>
    +{
    +  protected transient Class<?> clazz;
    + 
    +  @OutputPortFieldAnnotation
    +  public transient DefaultOutputPort<OUTPUT> out = new DefaultOutputPort<OUTPUT>();
    +  
    +  
    --- End diff --
    
    To many new lines.


---
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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41477002
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/JsonFormatter.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectWriter;
    +import org.codehaus.jackson.map.SerializationConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to JSON string
    + * <br><b>Properties</b>
    + *  <br><b>dateFormat</b>: date format
    + *  e.g dd/MM/yyyy
    + * @displayName JsonFormatter
    + * @category Formatter 
    + * @tags  pojo json formatter
    + */
    +public class JsonFormatter extends Formatter<String>
    +{
    +
    +  private transient ObjectWriter writer;
    +  protected String dateFormat;
    +  
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      if (dateFormat != null ) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      writer = mapper.writerWithType(clazz);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, true);
    +    } 
    +    catch (Throwable e) {
    +      throw new RuntimeException("Unable find provided class");
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    // TODO Auto-generated method stub
    --- End diff --
    
    Remove this?


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

Re: [GitHub] incubator-apex-malhar pull request: MLHR-1838 added parser and for...

Posted by Thomas Weise <th...@datatorrent.com>.
Hi Justin,

Thanks for checking. License header replacement is in the works:
https://github.com/apache/incubator-apex-malhar/pull/54

Will be in upcoming release and check will also be part of CI.

Thomas

On Wed, Oct 7, 2015 at 11:35 PM, Justin Mclean <ju...@classsoftware.com>
wrote:

> Hi,
>
> Copyright in the header should be the ASF not Data Torrent as the code has
> been donated to Apache. While this doesn’t have to be fixed right now, it
> might be worth getting it right for new files rather than fixing later.
>
> Thanks,
> Justin

Re: [GitHub] incubator-apex-malhar pull request: MLHR-1838 added parser and for...

Posted by Justin Mclean <ju...@classsoftware.com>.
Hi,

Copyright in the header should be the ASF not Data Torrent as the code has been donated to Apache. While this doesn’t have to be fixed right now, it might be worth getting it right for new files rather than fixing later.

Thanks,
Justin

[GitHub] incubator-apex-malhar pull request: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41479613
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,268 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] field_info = fieldInfo.split(",");
    +    for ( int i=0 ; i < field_info.length ; i++ ) {
    +      String[] field_tuple = field_info[i].split(":");
    +      Field field = new Field();
    +      field.setName(field_tuple[0]);
    +      String[] type_format = field_tuple[1].split("\\|");
    --- End diff --
    
    Thanks for clarifying!


---
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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41477563
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/JsonFormatter.java ---
    @@ -0,0 +1,99 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectWriter;
    +import org.codehaus.jackson.map.SerializationConfig;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to JSON string
    + * <br><b>Properties</b>
    + *  <br><b>dateFormat</b>: date format
    + *  e.g dd/MM/yyyy
    + * @displayName JsonFormatter
    + * @category Formatter 
    + * @tags  pojo json formatter
    + */
    +public class JsonFormatter extends Formatter<String>
    +{
    +
    +  private transient ObjectWriter writer;
    +  protected String dateFormat;
    +  
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      if (dateFormat != null ) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      writer = mapper.writerWithType(clazz);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_GETTERS, true);
    +      mapper.configure(SerializationConfig.Feature.AUTO_DETECT_IS_GETTERS, true);
    +    } 
    +    catch (Throwable e) {
    +      throw new RuntimeException("Unable find provided class");
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    // TODO Auto-generated method stub
    +    
    +  }
    +
    +  @Override
    +  public String convert(Object tuple)
    +  {
    +    try {
    +      return writer.writeValueAsString(tuple);
    +    }
    +    catch (Exception ex) {
    +      logger.error("error while deserializing tuple",ex.getMessage());
    --- End diff --
    
    Hi Thomas, the requirement was to  emit tuples that could not be parsed on error port. Hence, i chose to not propogate exception but rather to log them and emit the tuple on error port. Please guide if cases like these should be handled in a different way


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41665412
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/parser/CsvParser.java ---
    @@ -0,0 +1,314 @@
    +/**
    + * 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.schema.parser;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ParseBool;
    +import org.supercsv.cellprocessor.ParseChar;
    +import org.supercsv.cellprocessor.ParseDate;
    +import org.supercsv.cellprocessor.ParseDouble;
    +import org.supercsv.cellprocessor.ParseInt;
    +import org.supercsv.cellprocessor.ParseLong;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanReader;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.lib.util.ReusableStringReader;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts CSV string to Pojo <br>
    + * Assumption is that each field in the delimited data should map to a simple
    + * java type.<br>
    + * <br>
    + * <b>Properties</b> <br>
    + * <b>fieldInfo</b>:User need to specify fields and their types as a comma
    + * separated string having format &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in
    + * the same order as incoming data. FORMAT refers to dates with dd/mm/yyyy as
    + * default e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy <br>
    + * <b>fieldDelimiter</b>: Default is comma <br>
    + * <b>lineDelimiter</b>: Default is '\r\n'
    + * 
    + * @displayName CsvParser
    + * @category Parsers
    + * @tags csv pojo parser
    + */
    +@InterfaceStability.Evolving
    +public class CsvParser extends Parser<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +
    +  @NotNull
    +  protected String fieldInfo;
    +
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  private transient CsvBeanReader csvReader;
    +
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +
    +  @NotNull
    +  private transient ReusableStringReader csvStringReader = new ReusableStringReader();
    +
    +  public CsvParser()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +  }
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    logger.info("field info {}", fieldInfo);
    +    fields = new ArrayList<Field>();
    +    String[] fieldInfoTuple = fieldInfo.split(",");
    +    for (int i = 0; i < fieldInfoTuple.length; i++) {
    +      String[] fieldTuple = fieldInfoTuple[i].split(":");
    +      Field field = new Field();
    +      field.setName(fieldTuple[0]);
    +      String[] typeFormat = fieldTuple[1].split("\\|");
    +      field.setType(typeFormat[0].toUpperCase());
    +      if (typeFormat.length > 1) {
    +        field.setFormat(typeFormat[1]);
    +      }
    +      getFields().add(field);
    +    }
    +
    +    CsvPreference preference = new CsvPreference.Builder('"', fieldDelimiter, lineDelimiter).build();
    +    csvReader = new CsvBeanReader(csvStringReader, preference);
    +    int countKeyValue = getFields().size();
    +    logger.info("countKeyValue {}", countKeyValue);
    +    nameMapping = new String[countKeyValue];
    +    processors = new CellProcessor[countKeyValue];
    +    initialise(nameMapping, processors);
    +  }
    +
    +  private void initialise(String[] nameMapping, CellProcessor[] processors)
    +  {
    +    for (int i = 0; i < getFields().size(); i++) {
    +      FIELD_TYPE type = getFields().get(i).type;
    +      nameMapping[i] = getFields().get(i).name;
    +      if (type == FIELD_TYPE.DOUBLE) {
    +        processors[i] = new Optional(new ParseDouble());
    +      } else if (type == FIELD_TYPE.INTEGER) {
    +        processors[i] = new Optional(new ParseInt());
    +      } else if (type == FIELD_TYPE.FLOAT) {
    +        processors[i] = new Optional(new ParseDouble());
    +      } else if (type == FIELD_TYPE.LONG) {
    +        processors[i] = new Optional(new ParseLong());
    +      } else if (type == FIELD_TYPE.SHORT) {
    +        processors[i] = new Optional(new ParseInt());
    +      } else if (type == FIELD_TYPE.STRING) {
    +        processors[i] = new Optional();
    +      } else if (type == FIELD_TYPE.CHARACTER) {
    +        processors[i] = new Optional(new ParseChar());
    +      } else if (type == FIELD_TYPE.BOOLEAN) {
    +        processors[i] = new Optional(new ParseBool());
    +      } else if (type == FIELD_TYPE.DATE) {
    +        String dateFormat = getFields().get(i).format;
    +        processors[i] = new Optional(new ParseDate(dateFormat == null ? "dd/MM/yyyy" : dateFormat));
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +
    +  }
    +
    +  @Override
    +  public Object convert(String tuple)
    +  {
    +    try {
    +      csvStringReader.open(tuple);
    +      return csvReader.read(clazz, nameMapping, processors);
    +    } catch (IOException e) {
    +      logger.error("Error while converting tuple {} ", tuple + " " + e.getMessage());
    --- End diff --
    
    same as above comment for log message.


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599914
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/XmlFormatter.java ---
    @@ -0,0 +1,171 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.Writer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +
    +import com.thoughtworks.xstream.XStream;
    +import com.thoughtworks.xstream.XStreamException;
    +import com.thoughtworks.xstream.converters.basic.DateConverter;
    +import com.thoughtworks.xstream.io.HierarchicalStreamWriter;
    +import com.thoughtworks.xstream.io.xml.CompactWriter;
    +import com.thoughtworks.xstream.io.xml.XppDriver;
    +
    +/**
    + * @displayName XmlParser
    + * @category Formatter
    + * @tags xml pojo formatter
    + */
    +@InterfaceStability.Evolving
    +public class XmlFormatter extends Formatter<String>
    +{
    +
    +  private transient XStream xstream;
    +  
    +  protected String alias;
    +  protected String dateFormat;
    +  protected boolean prettyPrint;
    +  
    +  
    +  public XmlFormatter()
    +  {
    +    alias = null;
    +    dateFormat = null;
    +  }
    --- End diff --
    
    new line after }


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599901
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.exception.SuperCsvException;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +@InterfaceStability.Evolving
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] fieldInfoTuple = fieldInfo.split(",");
    +    for ( int i=0 ; i < fieldInfoTuple.length ; i++ ) {
    +      String[] fieldTuple = fieldInfoTuple[i].split(":");
    +      Field field = new Field();
    +      field.setName(fieldTuple[0]);
    +      String[] typeFormat = fieldTuple[1].split("\\|");
    +      field.setType(typeFormat[0].toUpperCase());
    +      if ( typeFormat.length > 1 ) {
    +        field.setFormat(typeFormat[1]);
    +      }
    +      getFields().add(field);
    +    }
    +    preference = new CsvPreference.Builder('"', fieldDelimiter, lineDelimiter).build();
    +    int countKeyValue = getFields().size();
    +    nameMapping = new String[countKeyValue];
    +    processors = new CellProcessor[countKeyValue];
    +    initialise(nameMapping, processors);
    +   
    +  }
    +
    +  private void initialise(String[] nameMapping, CellProcessor[] processors)
    +  {
    +    for (int i = 0; i < getFields().size(); i++) {
    +      FIELD_TYPE type = getFields().get(i).type;
    +      nameMapping[i] = getFields().get(i).name;
    +       if (type == FIELD_TYPE.DATE) {
    +        String dateFormat = getFields().get(i).format;
    +        processors[i] = new Optional(new FmtDate(dateFormat == null ? "dd/MM/yyyy" : dateFormat));
    +      }else{
    --- End diff --
    
    space between } and else and next {


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41599916
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/XmlFormatter.java ---
    @@ -0,0 +1,171 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.Writer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +
    +import com.thoughtworks.xstream.XStream;
    +import com.thoughtworks.xstream.XStreamException;
    +import com.thoughtworks.xstream.converters.basic.DateConverter;
    +import com.thoughtworks.xstream.io.HierarchicalStreamWriter;
    +import com.thoughtworks.xstream.io.xml.CompactWriter;
    +import com.thoughtworks.xstream.io.xml.XppDriver;
    +
    +/**
    + * @displayName XmlParser
    + * @category Formatter
    + * @tags xml pojo formatter
    + */
    +@InterfaceStability.Evolving
    +public class XmlFormatter extends Formatter<String>
    +{
    +
    +  private transient XStream xstream;
    +  
    +  protected String alias;
    +  protected String dateFormat;
    +  protected boolean prettyPrint;
    +  
    +  
    +  public XmlFormatter()
    +  {
    +    alias = null;
    +    dateFormat = null;
    +  }
    +  @Override
    +  public void activate(Context context)
    +  {
    +    if ( prettyPrint ) {
    +    xstream = new XStream();
    +    }else{
    --- End diff --
    
    spaces??


---
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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41478952
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/CsvFormatter.java ---
    @@ -0,0 +1,268 @@
    +/**
    + * Copyright (C) 2015 DataTorrent, Inc.
    + *
    + * Licensed 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.schema.formatter;
    +
    +import java.io.IOException;
    +import java.io.StringWriter;
    +import java.util.ArrayList;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.supercsv.cellprocessor.FmtDate;
    +import org.supercsv.cellprocessor.Optional;
    +import org.supercsv.cellprocessor.ift.CellProcessor;
    +import org.supercsv.io.CsvBeanWriter;
    +import org.supercsv.io.ICsvBeanWriter;
    +import org.supercsv.prefs.CsvPreference;
    +
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Operator that converts POJO to CSV string
    + * <br>Assumption is that each field in the delimited data should map to a simple java type.<br>
    + * <br><b>Properties</b>
    + * <br><b>fieldInfo</b>:User need to specify fields and their types as a comma separated string having format
    + *  &lt;NAME&gt;:&lt;TYPE&gt;|&lt;FORMAT&gt; in the same order as incoming data.
    + *  FORMAT refers to dates with dd/mm/yyyy as default
    + *  e.g name:string,dept:string,eid:integer,dateOfJoining:date|dd/mm/yyyy
    + * @displayName CsvFormatter
    + * @category Formatter 
    + * @tags pojo csv formatter
    + */
    +public class CsvFormatter extends Formatter<String>
    +{
    +
    +  private ArrayList<Field> fields;
    +  @NotNull
    +  protected String classname;
    +  @NotNull
    +  protected int fieldDelimiter;
    +  protected String lineDelimiter;
    +  
    +  @NotNull
    +  protected String fieldInfo;
    +  
    +  public enum FIELD_TYPE
    +  {
    +    BOOLEAN, DOUBLE, INTEGER, FLOAT, LONG, SHORT, CHARACTER, STRING, DATE
    +  };
    +  
    +  protected transient String[] nameMapping;
    +  protected transient CellProcessor[] processors;
    +  protected transient CsvPreference preference;
    +  
    +  public CsvFormatter()
    +  {
    +    fields = new ArrayList<Field>();
    +    fieldDelimiter = ',';
    +    lineDelimiter = "\r\n";
    +    
    +  }
    +  
    +  @Override public void setup(Context.OperatorContext context)
    +  {
    +    super.setup(context);
    +
    +    //fieldInfo information
    +    fields = new ArrayList<Field>();
    +    String[] field_info = fieldInfo.split(",");
    +    for ( int i=0 ; i < field_info.length ; i++ ) {
    +      String[] field_tuple = field_info[i].split(":");
    +      Field field = new Field();
    +      field.setName(field_tuple[0]);
    +      String[] type_format = field_tuple[1].split("\\|");
    --- End diff --
    
    String[] field_info = fieldInfo.split(",");
    
    Above line seems to have trouble deciding which style to use :-)
    
    We are using camel 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: MLHR-1838 added parser and for...

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

    https://github.com/apache/incubator-apex-malhar/pull/22#discussion_r41600637
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/formatter/XmlFormatter.java ---
    @@ -0,0 +1,171 @@
    +/**
    + * 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.schema.formatter;
    +
    +import java.io.Writer;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +
    +import com.thoughtworks.xstream.XStream;
    +import com.thoughtworks.xstream.XStreamException;
    +import com.thoughtworks.xstream.converters.basic.DateConverter;
    +import com.thoughtworks.xstream.io.HierarchicalStreamWriter;
    +import com.thoughtworks.xstream.io.xml.CompactWriter;
    +import com.thoughtworks.xstream.io.xml.XppDriver;
    +
    +/**
    + * @displayName XmlParser
    + * @category Formatter
    + * @tags xml pojo formatter
    + */
    +@InterfaceStability.Evolving
    +public class XmlFormatter extends Formatter<String>
    +{
    +
    +  private transient XStream xstream;
    +  
    +  protected String alias;
    +  protected String dateFormat;
    +  protected boolean prettyPrint;
    +  
    +  
    +  public XmlFormatter()
    +  {
    +    alias = null;
    +    dateFormat = null;
    +  }
    --- End diff --
    
    Formatted and committed 


---
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: MLHR-1838 added parser and for...

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/22#discussion_r41665429
  
    --- Diff: contrib/src/main/java/com/datatorrent/contrib/schema/parser/JsonParser.java ---
    @@ -0,0 +1,106 @@
    +/**
    + * 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.schema.parser;
    +
    +import java.io.IOException;
    +import java.text.SimpleDateFormat;
    +
    +import org.codehaus.jackson.JsonProcessingException;
    +import org.codehaus.jackson.map.DeserializationConfig;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.codehaus.jackson.map.ObjectReader;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.netlet.util.DTThrowable;
    +
    +/**
    + * Operator that converts JSON string to Pojo <br>
    + * <b>Properties</b> <br>
    + * <b>dateFormat</b>: date format e.g dd/MM/yyyy
    + * 
    + * @displayName JsonParser
    + * @category Parsers
    + * @tags json pojo parser
    + */
    +@InterfaceStability.Evolving
    +public class JsonParser extends Parser<String>
    +{
    +
    +  private transient ObjectReader reader;
    +  protected String dateFormat;
    +
    +  @Override
    +  public void activate(Context context)
    +  {
    +    try {
    +      ObjectMapper mapper = new ObjectMapper();
    +      mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    +      if (dateFormat != null) {
    +        mapper.setDateFormat(new SimpleDateFormat(dateFormat));
    +      }
    +      reader = mapper.reader(clazz);
    +    } catch (Throwable e) {
    +      throw new RuntimeException("Unable find provided class");
    +    }
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +  }
    +
    +  @Override
    +  public Object convert(String tuple)
    +  {
    +    try {
    +      return reader.readValue(tuple);
    +    } catch (JsonProcessingException e) {
    +      logger.error("Error while converting tuple {} ", tuple + " " + e.getMessage());
    --- End diff --
    
    same as above for log message.


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