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

hansva pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hop.git


The following commit(s) were added to refs/heads/master by this push:
     new beb1593  HOP-1963 : Can we read and write parquet data please?
     new 0748d52  Merge pull request #859 from mattcasters/master
beb1593 is described below

commit beb1593774ad8357ec165db91bb95a1917b9ae27
Author: Matt Casters <[email protected]>
AuthorDate: Wed Jun 9 16:56:23 2021 +0200

    HOP-1963 : Can we read and write parquet data please?
---
 assemblies/plugins/dist/pom.xml                    |  13 +
 assemblies/plugins/tech/parquet/pom.xml            |  81 ++++
 .../plugins/tech/parquet/src/assembly/assembly.xml | 108 +++++
 .../src/main/resources/version.xml}                |  27 +-
 assemblies/plugins/tech/pom.xml                    |   1 +
 .../pipeline/transforms/parquet-file-output.adoc   |  97 ++++
 plugins/tech/parquet/pom.xml                       | 123 +++++
 .../parquet/transforms/output/ParquetField.java    |  67 +++
 .../parquet/transforms/output/ParquetOutput.java   | 231 +++++++++
 .../transforms/output/ParquetOutputData.java       |  44 ++
 .../transforms/output/ParquetOutputDialog.java     | 536 +++++++++++++++++++++
 .../transforms/output/ParquetOutputFile.java       |  52 ++
 .../transforms/output/ParquetOutputMeta.java       | 345 +++++++++++++
 .../transforms/output/ParquetOutputStream.java     |  64 +++
 .../parquet/transforms/output/ParquetVersion.java  |  78 +++
 .../transforms/output/ParquetWriteSupport.java     | 102 ++++
 .../transforms/output/ParquetWriterBuilder.java    |  56 +++
 .../output/messages/messages_en_US.properties      |  37 ++
 .../tech/parquet/src/main/resources/parquet.svg    |  38 ++
 .../parquet/src/main/resources/parquet_input.svg   |  41 ++
 .../parquet/src/main/resources/parquet_output.svg  |  40 ++
 plugins/tech/pom.xml                               |   1 +
 22 files changed, 2156 insertions(+), 26 deletions(-)

diff --git a/assemblies/plugins/dist/pom.xml b/assemblies/plugins/dist/pom.xml
index e26b6a5..cdca5d0 100644
--- a/assemblies/plugins/dist/pom.xml
+++ b/assemblies/plugins/dist/pom.xml
@@ -3148,6 +3148,19 @@
 
     <dependency>
       <groupId>org.apache.hop</groupId>
+      <artifactId>hop-assemblies-plugins-tech-parquet</artifactId>
+      <version>1.0-SNAPSHOT</version>
+      <type>zip</type>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hop</groupId>
       <artifactId>hop-assemblies-plugins-misc-passwords</artifactId>
       <version>${hop-plugins-misc.version}</version>
       <type>zip</type>
diff --git a/assemblies/plugins/tech/parquet/pom.xml 
b/assemblies/plugins/tech/parquet/pom.xml
new file mode 100644
index 0000000..afd16f9
--- /dev/null
+++ b/assemblies/plugins/tech/parquet/pom.xml
@@ -0,0 +1,81 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.hop</groupId>
+    <artifactId>hop-assemblies-plugins-tech</artifactId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>hop-assemblies-plugins-tech-parquet</artifactId>
+  <version>1.0-SNAPSHOT</version>
+  <packaging>pom</packaging>
+  <name>Hop Assemblies Plugins Technology Parquet</name>
+  <description></description>
+
+  <properties>
+    <parquet.version>1.12.0</parquet.version>
+    <hadoop.version>2.6.5</hadoop.version>
+    <avro.version>1.8.2</avro.version>
+  </properties>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.hop</groupId>
+      <artifactId>hop-plugins-tech-parquet</artifactId>
+      <version>1.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-common</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-format-structures</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>
+
+  </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/assemblies/plugins/tech/parquet/src/assembly/assembly.xml 
b/assemblies/plugins/tech/parquet/src/assembly/assembly.xml
new file mode 100644
index 0000000..11bb700
--- /dev/null
+++ b/assemblies/plugins/tech/parquet/src/assembly/assembly.xml
@@ -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.
+  -->
+
+<assembly 
xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3";
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+          
xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3
 http://maven.apache.org/xsd/assembly-1.1.3.xsd";>
+  <id>hop-assemblies-plugins-tech-parquet</id>
+  <formats>
+    <format>zip</format>
+  </formats>
+  <baseDirectory>tech/parquet</baseDirectory>
+  <files>
+    <file>
+      <source>${project.basedir}/src/main/resources/version.xml</source>
+      <outputDirectory>.</outputDirectory>
+      <filtered>true</filtered>
+    </file>
+  </files>
+
+  <fileSets>
+    <fileSet>
+      <outputDirectory>lib</outputDirectory>
+      <excludes>
+        <exclude>**/*</exclude>
+      </excludes>
+    </fileSet>
+  </fileSets>
+  <dependencySets>
+    <dependencySet>
+      <useProjectArtifact>false</useProjectArtifact>
+      <includes>
+        <include>org.apache.hop:hop-plugins-tech-parquet:jar</include>
+      </includes>
+    </dependencySet>
+    <dependencySet>
+      <outputDirectory>lib</outputDirectory>
+      <useProjectArtifact>false</useProjectArtifact>
+      <scope>runtime</scope>
+      <includes>
+        <include>com.github.luben:zstd-jni:jar</include>
+        <include>com.google.code.findbugs:jsr305:jar</include>
+        <include>com.google.protobuf:protobuf-java:jar</include>
+        <include>commons-collections:commons-collections:jar</include>
+        <include>commons-digester:commons-digester:jar</include>
+        <include>commons-httpclient:commons-httpclient:jar</include>
+        <include>commons-net:commons-net:jar</include>
+        <include>com.sun.jersey:jersey-client:jar</include>
+        <include>com.sun.jersey:jersey-core:jar</include>
+        <include>io.netty:netty:jar</include>
+        <include>javax.activation:activation:jar</include>
+        <include>javax.xml.bind:jaxb-api:jar</include>
+        <include>javax.xml.stream:stax-api:jar</include>
+        <include>org.apache.avro:avro:jar</include>
+        <include>org.apache.curator:curator-client:jar</include>
+        <include>org.apache.curator:curator-framework:jar</include>
+        <include>org.apache.curator:curator-recipes:jar</include>
+        <include>org.apache.directory.api:api-asn1-api:jar</include>
+        <include>org.apache.directory.api:api-util:jar</include>
+        <include>org.apache.directory.server:apacheds-i18n:jar</include>
+        
<include>org.apache.directory.server:apacheds-kerberos-codec:jar</include>
+        <include>org.apache.hadoop:hadoop-annotations:jar</include>
+        <include>org.apache.hadoop:hadoop-auth:jar</include>
+        <include>org.apache.hadoop:hadoop-client:jar</include>
+        <include>org.apache.hadoop:hadoop-common:jar</include>
+        <include>org.apache.hadoop:hadoop-hdfs:jar</include>
+        <include>org.apache.hadoop:hadoop-mapreduce-client-app:jar</include>
+        <include>org.apache.hadoop:hadoop-mapreduce-client-common:jar</include>
+        <include>org.apache.hadoop:hadoop-mapreduce-client-core:jar</include>
+        
<include>org.apache.hadoop:hadoop-mapreduce-client-jobclient:jar</include>
+        
<include>org.apache.hadoop:hadoop-mapreduce-client-shuffle:jar</include>
+        <include>org.apache.hadoop:hadoop-yarn-api:jar</include>
+        <include>org.apache.hadoop:hadoop-yarn-client:jar</include>
+        <include>org.apache.hadoop:hadoop-yarn-common:jar</include>
+        <include>org.apache.hadoop:hadoop-yarn-server-common:jar</include>
+        <include>org.apache.parquet:parquet-avro:jar</include>
+        <include>org.apache.parquet:parquet-column:jar</include>
+        <include>org.apache.parquet:parquet-common:jar</include>
+        <include>org.apache.parquet:parquet-encoding:jar</include>
+        <include>org.apache.parquet:parquet-format-structures:jar</include>
+        <include>org.apache.parquet:parquet-hadoop:jar</include>
+        <include>org.apache.parquet:parquet-jackson:jar</include>
+        <include>org.apache.yetus:audience-annotations:jar</include>
+        <include>org.apache.zookeeper:zookeeper:jar</include>
+        <include>org.codehaus.jackson:jackson-core-asl:jar</include>
+        <include>org.codehaus.jackson:jackson-jaxrs:jar</include>
+        <include>org.codehaus.jackson:jackson-mapper-asl:jar</include>
+        <include>org.codehaus.jackson:jackson-xc:jar</include>
+        <include>org.fusesource.leveldbjni:leveldbjni-all:jar</include>
+        <include>org.htrace:htrace-core:jar</include>
+        <include>xmlenc:xmlenc:jar</include>
+      </includes>
+    </dependencySet>
+  </dependencySets>
+</assembly>
diff --git a/assemblies/plugins/tech/pom.xml 
b/assemblies/plugins/tech/parquet/src/main/resources/version.xml
similarity index 53%
copy from assemblies/plugins/tech/pom.xml
copy to assemblies/plugins/tech/parquet/src/main/resources/version.xml
index 3cbef90..ee1c237 100644
--- a/assemblies/plugins/tech/pom.xml
+++ b/assemblies/plugins/tech/parquet/src/main/resources/version.xml
@@ -16,29 +16,4 @@
   ~ limitations under the License.
   -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0";
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
-  <modelVersion>4.0.0</modelVersion>
-
-  <parent>
-    <artifactId>hop-assemblies-plugins</artifactId>
-    <groupId>org.apache.hop</groupId>
-    <version>1.0-SNAPSHOT</version>
-  </parent>
-  <packaging>pom</packaging>
-
-  <artifactId>hop-assemblies-plugins-tech</artifactId>
-
-  <name>Hop Assemblies Plugins Technology</name>
-  <description></description>
-
-  <modules>
-    <module>avro</module>
-    <module>azure</module>
-    <module>cassandra</module>
-    <module>google</module>
-    <module>neo4j</module>
-  </modules>
-
-</project>
\ No newline at end of file
+<version>${project.version}</version>
\ No newline at end of file
diff --git a/assemblies/plugins/tech/pom.xml b/assemblies/plugins/tech/pom.xml
index 3cbef90..11db5af 100644
--- a/assemblies/plugins/tech/pom.xml
+++ b/assemblies/plugins/tech/pom.xml
@@ -39,6 +39,7 @@
     <module>cassandra</module>
     <module>google</module>
     <module>neo4j</module>
+    <module>parquet</module>
   </modules>
 
 </project>
\ No newline at end of file
diff --git 
a/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-output.adoc
 
b/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-output.adoc
new file mode 100644
index 0000000..b839390
--- /dev/null
+++ 
b/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-output.adoc
@@ -0,0 +1,97 @@
+////
+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.
+////
+:documentationPath: /pipeline/transforms/
+:language: en_US
+
+= Parquet File Output
+
+== Description
+
+This transform is capable of writing data into the Apache Parquet file format.
+For more information on this see: http://parquet.apache.org/[Apache Parquet].
+
+== Options
+
+Notes:
+
+* The date optionally referenced in the output file name(s) will be the start 
of the pipeline execution.
+* Hop Date types are serialized as EPOC: milliseconds since `1970-01-01 
00:00:00.000`
+* Strings are written as binary in UTF-8
+* Compression of data into columnar format is being done in memory.
+This happens when all rows are written.
+To not run out of memory make sure to specify a split size.
+
+[width="90%",options="header"]
+|===
+|Option|Description
+
+|Transform name
+|Name of the transform this name has to be unique in a single pipeline.
+
+|Base file name
+|Specify the base filename.
+This is composed of where you want to write the Parquet file to as well as the 
start of the filename.
+Examples:
+
+Write to Amazon AWS S3 : `s3://my-bucket-name/transactions`
+
+Write to a local folder : `/my/folder/customer-data`
+
+|Extension
+|This is the extension of the file.
+Usually this is simply `snappy`
+
+|Include date?
+|Check this box if you want to include the date in the filename with mask 
`yyyMMdd`
+
+|Include time?
+|Check this box if you want to include the time in the filename with mask 
`HHmmss`
+
+|Include date-time-format?
+|Check this box if you want to include a specific custom date-time format in 
the filename
+
+|Include transform copy number?
+|Enable this option if you run this transform in multiple copies to not have 
multiple threads write to the same file.
+The copy number is formatted with mask `00`
+
+|Split into parts and include number?
+|Enable this option if you want to split the output into multiple parts.
+Specify a split size larger than 0 and this is then the number of rows per 
file.
+The file part (split) number will be included in the filename to make sure 
that the same file is not being overwritten.
+The split number is formatted with mask `0000`
+
+|Compression codec
+|Here you can indicate which compression codec you want to use.
+The default is SNAPPY for Apache Snappy compression.
+
+|Version
+|Choose the protocol version of Parquet (1.0 or 2.0)
+
+|Row group size
+|The amount of rows in a group
+
+|Data page size
+|The data page size on a 1kB boundary (default is 1048576)
+
+|Dictionary page size
+|The data dictionary page size on a 1kB boundary (default is 1048576)
+
+|Fields
+|You can specify which fields to write and in which order.
+You can use the "Get Fields" button to populate the dialog.
+
+|===
\ No newline at end of file
diff --git a/plugins/tech/parquet/pom.xml b/plugins/tech/parquet/pom.xml
new file mode 100755
index 0000000..bb0c179
--- /dev/null
+++ b/plugins/tech/parquet/pom.xml
@@ -0,0 +1,123 @@
+<?xml version="1.0"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
+  <modelVersion>4.0.0</modelVersion>
+
+  <groupId>org.apache.hop</groupId>
+  <artifactId>hop-plugins-tech-parquet</artifactId>
+  <version>1.0-SNAPSHOT</version>
+  <packaging>jar</packaging>
+
+  <name>Hop Plugins Technology Parquet</name>
+
+  <parent>
+    <groupId>org.apache.hop</groupId>
+    <artifactId>hop-plugins-tech</artifactId>
+    <version>1.0-SNAPSHOT</version>
+  </parent>
+
+  <licenses>
+    <license>
+      <name>Apache License, version 2.0</name>
+      <url>https://www.apache.org/licenses/LICENSE-2.0</url>
+      <distribution>repo</distribution>
+      <comments/>
+    </license>
+  </licenses>
+
+  <properties>
+    <parquet.version>1.12.0</parquet.version>
+    <hadoop.version>2.6.5</hadoop.version>
+    <avro.version>1.8.2</avro.version>
+  </properties>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-common</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-format-structures</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-column</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <version>${hadoop.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hop</groupId>
+      <artifactId>hop-core</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>${mockito-all.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hop</groupId>
+      <artifactId>hop-ui</artifactId>
+      <version>${project.version}</version>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hop</groupId>
+      <artifactId>hop-engine</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hop</groupId>
+      <artifactId>hop-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
+</project>
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetField.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetField.java
new file mode 100644
index 0000000..1d60c2e
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetField.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hop.metadata.api.HopMetadataProperty;
+
+public class ParquetField {
+  @HopMetadataProperty(key = "source_field")
+  private String sourceFieldName;
+
+  @HopMetadataProperty(key = "target_field")
+  private String targetFieldName;
+
+  public ParquetField() {}
+
+  public ParquetField(String sourceFieldName, String targetFieldName) {
+    this.sourceFieldName = sourceFieldName;
+    this.targetFieldName = targetFieldName;
+  }
+
+  public ParquetField(ParquetField f) {
+    this(f.sourceFieldName, f.targetFieldName);
+  }
+
+  /**
+   * Gets sourceFieldName
+   *
+   * @return value of sourceFieldName
+   */
+  public String getSourceFieldName() {
+    return sourceFieldName;
+  }
+
+  /** @param sourceFieldName The sourceFieldName to set */
+  public void setSourceFieldName(String sourceFieldName) {
+    this.sourceFieldName = sourceFieldName;
+  }
+
+  /**
+   * Gets targetFieldName
+   *
+   * @return value of targetFieldName
+   */
+  public String getTargetFieldName() {
+    return targetFieldName;
+  }
+
+  /** @param targetFieldName The targetFieldName to set */
+  public void setTargetFieldName(String targetFieldName) {
+    this.targetFieldName = targetFieldName;
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutput.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutput.java
new file mode 100644
index 0000000..0d3cc91
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutput.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hop.core.Const;
+import org.apache.hop.core.RowMetaAndData;
+import org.apache.hop.core.exception.HopException;
+import org.apache.hop.core.row.IValueMeta;
+import org.apache.hop.core.vfs.HopVfs;
+import org.apache.hop.pipeline.Pipeline;
+import org.apache.hop.pipeline.PipelineMeta;
+import org.apache.hop.pipeline.transform.BaseTransform;
+import org.apache.hop.pipeline.transform.ITransform;
+import org.apache.hop.pipeline.transform.TransformMeta;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+
+import java.text.DecimalFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+public class ParquetOutput extends BaseTransform<ParquetOutputMeta, 
ParquetOutputData>
+    implements ITransform<ParquetOutputMeta, ParquetOutputData> {
+
+  public ParquetOutput(
+      TransformMeta transformMeta,
+      ParquetOutputMeta meta,
+      ParquetOutputData data,
+      int copyNr,
+      PipelineMeta pipelineMeta,
+      Pipeline pipeline) {
+    super(transformMeta, meta, data, copyNr, pipelineMeta, pipeline);
+  }
+
+  @Override
+  public boolean init() {
+
+    // Pre-calculate some values...
+    //
+    data.pageSize =
+        Const.toInt(resolve(meta.getDataPageSize()), 
ParquetProperties.DEFAULT_PAGE_SIZE);
+    data.dictionaryPageSize =
+        Const.toInt(
+            resolve(meta.getDictionaryPageSize()), 
ParquetProperties.DEFAULT_DICTIONARY_PAGE_SIZE);
+    data.rowGroupSize =
+        Const.toInt(
+            resolve(meta.getRowGroupSize()), 
ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT);
+    data.maxSplitSizeRows = Const.toLong(resolve(meta.getFileSplitSize()), -1);
+
+    return super.init();
+  }
+
+  @Override
+  public boolean processRow() throws HopException {
+    Object[] row = getRow();
+    if (row == null) {
+      closeFile();
+      setOutputDone();
+      return false;
+    }
+
+    if (first) {
+      first = false;
+      data.sourceFieldIndexes = new ArrayList<>();
+      for (int i = 0; i < meta.getFields().size(); i++) {
+        ParquetField field = meta.getFields().get(i);
+        int index = getInputRowMeta().indexOfValue(field.getSourceFieldName());
+        if (index < 0) {
+          throw new HopException("Unable to find source field '" + 
field.getSourceFieldName());
+        }
+        data.sourceFieldIndexes.add(index);
+      }
+
+      openNewFile();
+    }
+
+    // See if we don't need to create a new file split into parts...
+    //
+    if (meta.isFilenameIncludingSplitNr()
+        && data.maxSplitSizeRows > 0
+        && data.splitRowCount >= data.maxSplitSizeRows) {
+      // Close file and start a new one...
+      //
+      closeFile();
+      openNewFile();
+    }
+
+    // Write the row, handled by class ParquetWriteSupport
+    //
+    try {
+      data.writer.write(new RowMetaAndData(getInputRowMeta(), row));
+      incrementLinesOutput();
+      data.splitRowCount++;
+    } catch (Exception e) {
+      throw new HopException("Error writing row to parquet file", e);
+    }
+
+    putRow(getInputRowMeta(), row);
+    return true;
+  }
+
+  private void openNewFile() throws HopException {
+
+    data.splitRowCount = 0;
+    data.split++;
+
+    // Hadoop configuration
+    //
+    data.conf = new Configuration();
+
+    // Parquet Properties
+    //
+    ParquetProperties.Builder builder = ParquetProperties.builder();
+    switch (meta.getVersion()) {
+      case Version1:
+        builder = 
builder.withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0);
+        break;
+      case Version2:
+        builder = 
builder.withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0);
+        break;
+    }
+    data.props = builder.build();
+
+    List<Type> types = new ArrayList<>();
+
+    // Build the Parquet Schema
+    //
+    for (int i = 0; i < meta.getFields().size(); i++) {
+      ParquetField field = meta.getFields().get(i);
+      IValueMeta valueMeta = 
getInputRowMeta().getValueMeta(data.sourceFieldIndexes.get(i));
+
+      PrimitiveType.PrimitiveTypeName typeName = 
PrimitiveType.PrimitiveTypeName.BINARY;
+
+      switch (valueMeta.getType()) {
+        case IValueMeta.TYPE_DATE:
+        case IValueMeta.TYPE_INTEGER:
+          typeName = PrimitiveType.PrimitiveTypeName.INT64;
+          break;
+        case IValueMeta.TYPE_NUMBER:
+          typeName = PrimitiveType.PrimitiveTypeName.DOUBLE;
+          break;
+        case IValueMeta.TYPE_BOOLEAN:
+          typeName = PrimitiveType.PrimitiveTypeName.BOOLEAN;
+          break;
+        case IValueMeta.TYPE_BINARY:
+          typeName = PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+          break;
+      }
+      Type type = new PrimitiveType(Type.Repetition.REQUIRED, typeName, 
field.getTargetFieldName());
+      types.add(type);
+    }
+    MessageType messageType = new MessageType("HopParquetSchema", types);
+
+    // Calculate the filename...
+    //
+    data.filename = buildFilename(getPipeline().getExecutionStartDate());
+
+    try {
+      data.outputStream = HopVfs.getOutputStream(data.filename, false);
+      data.outputFile = new ParquetOutputFile(data.outputStream);
+
+      data.writer =
+          new ParquetWriterBuilder(
+                  messageType, data.outputFile, data.sourceFieldIndexes, 
meta.getFields())
+              .withPageSize(data.pageSize)
+              .withDictionaryPageSize(data.dictionaryPageSize)
+              .withValidation(ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED)
+              .withCompressionCodec(meta.getCompressionCodec())
+              .withRowGroupSize(data.rowGroupSize)
+              .withWriterVersion(data.props.getWriterVersion())
+              .withWriteMode(ParquetFileWriter.Mode.CREATE)
+              .build();
+
+    } catch (Exception e) {
+      throw new HopException("Unable to create output file '" + data.filename 
+ "'", e);
+    }
+  }
+
+  private String buildFilename(Date date) {
+    String filename = resolve(meta.getFilenameBase());
+    if (meta.isFilenameIncludingDate()) {
+      filename += "-" + new SimpleDateFormat("yyyyMMdd").format(date);
+    }
+    if (meta.isFilenameIncludingTime()) {
+      filename += "-" + new SimpleDateFormat("HHmmss").format(date);
+    }
+    if (meta.isFilenameIncludingDateTime()) {
+      filename +=
+          "-" + new 
SimpleDateFormat(resolve(meta.getFilenameDateTimeFormat())).format(date);
+    }
+    if (meta.isFilenameIncludingCopyNr()) {
+      filename += "-" + new DecimalFormat("00").format(getCopyNr());
+    }
+    if (meta.isFilenameIncludingSplitNr()) {
+      filename += "-" + new DecimalFormat("0000").format(data.split);
+    }
+    filename += "." + Const.NVL(resolve(meta.getFilenameExtension()), 
"parquet");
+    filename += meta.getCompressionCodec().getExtension();
+    return filename;
+  }
+
+  private void closeFile() throws HopException {
+    try {
+      data.writer.close();
+    } catch (Exception e) {
+      throw new HopException("Error closing file " + data.filename, e);
+    }
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputData.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputData.java
new file mode 100644
index 0000000..d02f963
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputData.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hop.core.RowMetaAndData;
+import org.apache.hop.pipeline.transform.BaseTransformData;
+import org.apache.hop.pipeline.transform.ITransformData;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.ParquetWriter;
+
+import java.io.OutputStream;
+import java.util.ArrayList;
+
+public class ParquetOutputData extends BaseTransformData implements 
ITransformData {
+  public ArrayList<Integer> sourceFieldIndexes;
+  public Configuration conf;
+  public ParquetProperties props;
+  public String filename;
+  public OutputStream outputStream;
+  public ParquetOutputFile outputFile;
+  public ParquetWriter<RowMetaAndData> writer;
+  public int split = 0;
+  public long splitRowCount;
+  public long maxSplitSizeRows;
+  public int rowGroupSize;
+  public int pageSize;
+  public int dictionaryPageSize;
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputDialog.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputDialog.java
new file mode 100644
index 0000000..f37d19d
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputDialog.java
@@ -0,0 +1,536 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hop.core.Const;
+import org.apache.hop.core.logging.LogChannel;
+import org.apache.hop.core.row.IRowMeta;
+import org.apache.hop.core.variables.IVariables;
+import org.apache.hop.i18n.BaseMessages;
+import org.apache.hop.pipeline.PipelineMeta;
+import org.apache.hop.pipeline.transform.BaseTransformMeta;
+import org.apache.hop.pipeline.transform.ITransformDialog;
+import org.apache.hop.ui.core.dialog.BaseDialog;
+import org.apache.hop.ui.core.dialog.ErrorDialog;
+import org.apache.hop.ui.core.gui.WindowProperty;
+import org.apache.hop.ui.core.widget.ColumnInfo;
+import org.apache.hop.ui.core.widget.TableView;
+import org.apache.hop.ui.core.widget.TextVar;
+import org.apache.hop.ui.pipeline.transform.BaseTransformDialog;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.eclipse.swt.SWT;
+import org.eclipse.swt.layout.FormAttachment;
+import org.eclipse.swt.layout.FormData;
+import org.eclipse.swt.layout.FormLayout;
+import org.eclipse.swt.widgets.Button;
+import org.eclipse.swt.widgets.Combo;
+import org.eclipse.swt.widgets.Control;
+import org.eclipse.swt.widgets.Group;
+import org.eclipse.swt.widgets.Label;
+import org.eclipse.swt.widgets.Shell;
+import org.eclipse.swt.widgets.TableItem;
+import org.eclipse.swt.widgets.Text;
+
+public class ParquetOutputDialog extends BaseTransformDialog implements 
ITransformDialog {
+
+  public static final Class<?> PKG = ParquetOutputMeta.class;
+
+  protected ParquetOutputMeta input;
+
+  private Shell shell;
+  private TextVar wFilenameBase;
+  private TextVar wFilenameExtension;
+  private Button wFilenameIncludeDate;
+  private Button wFilenameIncludeTime;
+  private Button wFilenameIncludeDateTime;
+  private Label wlFilenameDateTimeFormat;
+  private TextVar wFilenameDateTimeFormat;
+  private Button wFilenameIncludeCopyNr;
+  private Button wFilenameIncludeSplitNr;
+  private Label wlFilenameSplitSize;
+  private TextVar wFilenameSplitSize;
+  private Combo wCompressionCodec;
+  private Combo wVersion;
+  private TextVar wRowGroupSize;
+  private TextVar wDataPageSize;
+  private TextVar wDictionaryPageSize;
+  private TableView wFields;
+
+  private String returnValue;
+
+  public ParquetOutputDialog(
+      Shell parent,
+      IVariables variables,
+      Object in,
+      PipelineMeta pipelineMeta,
+      String transformName) {
+    super(parent, variables, (BaseTransformMeta) in, pipelineMeta, 
transformName);
+    input = (ParquetOutputMeta) in;
+  }
+
+  @Override
+  public String open() {
+
+    Shell parent = getParent();
+
+    shell = new Shell(parent, SWT.DIALOG_TRIM | SWT.RESIZE | SWT.MIN | 
SWT.MAX);
+    props.setLook(shell);
+    setShellImage(shell, input);
+
+    FormLayout formLayout = new FormLayout();
+    formLayout.marginWidth = Const.FORM_MARGIN;
+    formLayout.marginHeight = Const.FORM_MARGIN;
+
+    shell.setLayout(formLayout);
+    shell.setText(BaseMessages.getString(PKG, "ParquetOutput.Name"));
+
+    int middle = props.getMiddlePct();
+    int margin = props.getMargin();
+
+    // Some buttons at the bottom
+    wOk = new Button(shell, SWT.PUSH);
+    wOk.setText(BaseMessages.getString(PKG, "System.Button.OK"));
+    wOk.addListener(SWT.Selection, e -> ok());
+    wGet = new Button(shell, SWT.PUSH);
+    wGet.setText(BaseMessages.getString(PKG, "System.Button.GetFields"));
+    wGet.addListener(SWT.Selection, e -> getFields());
+    wCancel = new Button(shell, SWT.PUSH);
+    wCancel.setText(BaseMessages.getString(PKG, "System.Button.Cancel"));
+    wCancel.addListener(SWT.Selection, e -> cancel());
+    setButtonPositions(new Button[] {wOk, wGet, wCancel}, margin, null);
+
+    // TransformName line
+    wlTransformName = new Label(shell, SWT.RIGHT);
+    wlTransformName.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.TransformName.Label"));
+    props.setLook(wlTransformName);
+    fdlTransformName = new FormData();
+    fdlTransformName.left = new FormAttachment(0, 0);
+    fdlTransformName.right = new FormAttachment(middle, -margin);
+    fdlTransformName.top = new FormAttachment(0, margin);
+    wlTransformName.setLayoutData(fdlTransformName);
+    wTransformName = new Text(shell, SWT.SINGLE | SWT.LEFT | SWT.BORDER);
+    wTransformName.setText(transformName);
+    props.setLook(wTransformName);
+    fdTransformName = new FormData();
+    fdTransformName.left = new FormAttachment(middle, 0);
+    fdTransformName.top = new FormAttachment(wlTransformName, 0, SWT.CENTER);
+    fdTransformName.right = new FormAttachment(100, 0);
+    wTransformName.setLayoutData(fdTransformName);
+    Control lastControl = wTransformName;
+
+    Group wFileGroup = new Group(shell, SWT.SHADOW_ETCHED_IN);
+    wFileGroup.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameGroup.Label"));
+    wFileGroup.setLayout(new FormLayout());
+    FormData fdFileGroup = new FormData();
+    fdFileGroup.left = new FormAttachment(0, 0);
+    fdFileGroup.right = new FormAttachment(100, 0);
+    fdFileGroup.top = new FormAttachment(lastControl, margin);
+    wFileGroup.setLayoutData(fdFileGroup);
+
+    Label wlFilenameBase = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameBase.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameBase.Label"));
+    props.setLook(wlFilenameBase);
+    FormData fdlFilenameBase = new FormData();
+    fdlFilenameBase.left = new FormAttachment(0, 0);
+    fdlFilenameBase.right = new FormAttachment(middle, -margin);
+    fdlFilenameBase.top = new FormAttachment(0, 0);
+    wlFilenameBase.setLayoutData(fdlFilenameBase);
+    wFilenameBase = new TextVar(variables, wFileGroup, SWT.SINGLE | SWT.LEFT | 
SWT.BORDER);
+    props.setLook(wFilenameBase);
+    FormData fdFilenameBase = new FormData();
+    fdFilenameBase.left = new FormAttachment(middle, 0);
+    fdFilenameBase.top = new FormAttachment(wlFilenameBase, 0, SWT.CENTER);
+    fdFilenameBase.right = new FormAttachment(100, 0);
+    wFilenameBase.setLayoutData(fdFilenameBase);
+    lastControl = wFilenameBase;
+
+    Label wlFilenameExtension = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameExtension.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameExtension.Label"));
+    props.setLook(wlFilenameExtension);
+    FormData fdlFilenameExtension = new FormData();
+    fdlFilenameExtension.left = new FormAttachment(0, 0);
+    fdlFilenameExtension.right = new FormAttachment(middle, -margin);
+    fdlFilenameExtension.top = new FormAttachment(lastControl, margin);
+    wlFilenameExtension.setLayoutData(fdlFilenameExtension);
+    wFilenameExtension = new TextVar(variables, wFileGroup, SWT.SINGLE | 
SWT.LEFT | SWT.BORDER);
+    props.setLook(wFilenameExtension);
+    FormData fdFilenameExtension = new FormData();
+    fdFilenameExtension.left = new FormAttachment(middle, 0);
+    fdFilenameExtension.top = new FormAttachment(wlFilenameExtension, 0, 
SWT.CENTER);
+    fdFilenameExtension.right = new FormAttachment(100, 0);
+    wFilenameExtension.setLayoutData(fdFilenameExtension);
+    lastControl = wFilenameExtension;
+
+    Label wlFilenameIncludeDate = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameIncludeDate.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameIncludeDate.Label"));
+    props.setLook(wlFilenameIncludeDate);
+    FormData fdlFilenameIncludeDate = new FormData();
+    fdlFilenameIncludeDate.left = new FormAttachment(0, 0);
+    fdlFilenameIncludeDate.right = new FormAttachment(middle, -margin);
+    fdlFilenameIncludeDate.top = new FormAttachment(lastControl, margin);
+    wlFilenameIncludeDate.setLayoutData(fdlFilenameIncludeDate);
+    wFilenameIncludeDate = new Button(wFileGroup, SWT.CHECK);
+    props.setLook(wFilenameIncludeDate);
+    FormData fdFilenameIncludeDate = new FormData();
+    fdFilenameIncludeDate.left = new FormAttachment(middle, 0);
+    fdFilenameIncludeDate.top = new FormAttachment(wlFilenameIncludeDate, 0, 
SWT.CENTER);
+    fdFilenameIncludeDate.right = new FormAttachment(100, 0);
+    wFilenameIncludeDate.setLayoutData(fdFilenameIncludeDate);
+    lastControl = wlFilenameIncludeDate;
+
+    Label wlFilenameIncludeTime = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameIncludeTime.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameIncludeTime.Label"));
+    props.setLook(wlFilenameIncludeTime);
+    FormData fdlFilenameIncludeTime = new FormData();
+    fdlFilenameIncludeTime.left = new FormAttachment(0, 0);
+    fdlFilenameIncludeTime.right = new FormAttachment(middle, -margin);
+    fdlFilenameIncludeTime.top = new FormAttachment(lastControl, margin);
+    wlFilenameIncludeTime.setLayoutData(fdlFilenameIncludeTime);
+    wFilenameIncludeTime = new Button(wFileGroup, SWT.CHECK);
+    props.setLook(wFilenameIncludeTime);
+    FormData fdFilenameIncludeTime = new FormData();
+    fdFilenameIncludeTime.left = new FormAttachment(middle, 0);
+    fdFilenameIncludeTime.top = new FormAttachment(wlFilenameIncludeTime, 0, 
SWT.CENTER);
+    fdFilenameIncludeTime.right = new FormAttachment(100, 0);
+    wFilenameIncludeTime.setLayoutData(fdFilenameIncludeTime);
+    lastControl = wlFilenameIncludeTime;
+
+    Label wlFilenameIncludeDateTime = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameIncludeDateTime.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameIncludeDateTime.Label"));
+    props.setLook(wlFilenameIncludeDateTime);
+    FormData fdlFilenameIncludeDateTime = new FormData();
+    fdlFilenameIncludeDateTime.left = new FormAttachment(0, 0);
+    fdlFilenameIncludeDateTime.right = new FormAttachment(middle, -margin);
+    fdlFilenameIncludeDateTime.top = new FormAttachment(lastControl, margin);
+    wlFilenameIncludeDateTime.setLayoutData(fdlFilenameIncludeDateTime);
+    wFilenameIncludeDateTime = new Button(wFileGroup, SWT.CHECK);
+    props.setLook(wFilenameIncludeDateTime);
+    FormData fdFilenameIncludeDateTime = new FormData();
+    fdFilenameIncludeDateTime.left = new FormAttachment(middle, 0);
+    fdFilenameIncludeDateTime.top = new 
FormAttachment(wlFilenameIncludeDateTime, 0, SWT.CENTER);
+    fdFilenameIncludeDateTime.right = new FormAttachment(100, 0);
+    wFilenameIncludeDateTime.setLayoutData(fdFilenameIncludeDateTime);
+    wFilenameIncludeDateTime.addListener(SWT.Selection, e -> enableFields());
+    lastControl = wlFilenameIncludeDateTime;
+
+    wlFilenameDateTimeFormat = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameDateTimeFormat.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameDateTimeFormat.Label"));
+    props.setLook(wlFilenameDateTimeFormat);
+    FormData fdlFilenameDateTimeFormat = new FormData();
+    fdlFilenameDateTimeFormat.left = new FormAttachment(0, 0);
+    fdlFilenameDateTimeFormat.right = new FormAttachment(middle, -margin);
+    fdlFilenameDateTimeFormat.top = new FormAttachment(lastControl, margin);
+    wlFilenameDateTimeFormat.setLayoutData(fdlFilenameDateTimeFormat);
+    wFilenameDateTimeFormat =
+        new TextVar(variables, wFileGroup, SWT.SINGLE | SWT.LEFT | SWT.BORDER);
+    props.setLook(wFilenameDateTimeFormat);
+    FormData fdFilenameDateTimeFormat = new FormData();
+    fdFilenameDateTimeFormat.left = new FormAttachment(middle, 0);
+    fdFilenameDateTimeFormat.top = new 
FormAttachment(wlFilenameDateTimeFormat, 0, SWT.CENTER);
+    fdFilenameDateTimeFormat.right = new FormAttachment(100, 0);
+    wFilenameDateTimeFormat.setLayoutData(fdFilenameDateTimeFormat);
+    lastControl = wFilenameDateTimeFormat;
+
+    Label wlFilenameIncludeCopyNr = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameIncludeCopyNr.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameIncludeCopyNr.Label"));
+    props.setLook(wlFilenameIncludeCopyNr);
+    FormData fdlFilenameIncludeCopyNr = new FormData();
+    fdlFilenameIncludeCopyNr.left = new FormAttachment(0, 0);
+    fdlFilenameIncludeCopyNr.right = new FormAttachment(middle, -margin);
+    fdlFilenameIncludeCopyNr.top = new FormAttachment(lastControl, margin);
+    wlFilenameIncludeCopyNr.setLayoutData(fdlFilenameIncludeCopyNr);
+    wFilenameIncludeCopyNr = new Button(wFileGroup, SWT.CHECK);
+    props.setLook(wFilenameIncludeCopyNr);
+    FormData fdFilenameIncludeCopyNr = new FormData();
+    fdFilenameIncludeCopyNr.left = new FormAttachment(middle, 0);
+    fdFilenameIncludeCopyNr.top = new FormAttachment(wlFilenameIncludeCopyNr, 
0, SWT.CENTER);
+    fdFilenameIncludeCopyNr.right = new FormAttachment(100, 0);
+    wFilenameIncludeCopyNr.setLayoutData(fdFilenameIncludeCopyNr);
+    lastControl = wlFilenameIncludeCopyNr;
+
+    Label wlFilenameIncludeSplitNr = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameIncludeSplitNr.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameIncludeSplitNr.Label"));
+    props.setLook(wlFilenameIncludeSplitNr);
+    FormData fdlFilenameIncludeSplitNr = new FormData();
+    fdlFilenameIncludeSplitNr.left = new FormAttachment(0, 0);
+    fdlFilenameIncludeSplitNr.right = new FormAttachment(middle, -margin);
+    fdlFilenameIncludeSplitNr.top = new FormAttachment(lastControl, margin);
+    wlFilenameIncludeSplitNr.setLayoutData(fdlFilenameIncludeSplitNr);
+    wFilenameIncludeSplitNr = new Button(wFileGroup, SWT.CHECK);
+    props.setLook(wFilenameIncludeSplitNr);
+    FormData fdFilenameIncludeSplitNr = new FormData();
+    fdFilenameIncludeSplitNr.left = new FormAttachment(middle, 0);
+    fdFilenameIncludeSplitNr.top = new 
FormAttachment(wlFilenameIncludeSplitNr, 0, SWT.CENTER);
+    fdFilenameIncludeSplitNr.right = new FormAttachment(100, 0);
+    wFilenameIncludeSplitNr.setLayoutData(fdFilenameIncludeSplitNr);
+    wFilenameIncludeSplitNr.addListener(SWT.Selection, e -> enableFields());
+    lastControl = wlFilenameIncludeSplitNr;
+
+    wlFilenameSplitSize = new Label(wFileGroup, SWT.RIGHT);
+    wlFilenameSplitSize.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.FilenameSplitSize.Label"));
+    props.setLook(wlFilenameSplitSize);
+    FormData fdlFilenameSplitSize = new FormData();
+    fdlFilenameSplitSize.left = new FormAttachment(0, 0);
+    fdlFilenameSplitSize.right = new FormAttachment(middle, -margin);
+    fdlFilenameSplitSize.top = new FormAttachment(lastControl, margin);
+    wlFilenameSplitSize.setLayoutData(fdlFilenameSplitSize);
+    wFilenameSplitSize = new TextVar(variables, wFileGroup, SWT.SINGLE | 
SWT.LEFT | SWT.BORDER);
+    props.setLook(wFilenameSplitSize);
+    FormData fdFilenameSplitSize = new FormData();
+    fdFilenameSplitSize.left = new FormAttachment(middle, 0);
+    fdFilenameSplitSize.top = new FormAttachment(wlFilenameSplitSize, 0, 
SWT.CENTER);
+    fdFilenameSplitSize.right = new FormAttachment(100, 0);
+    wFilenameSplitSize.setLayoutData(fdFilenameSplitSize);
+
+    // End of the file group
+    //
+    lastControl = wFileGroup;
+
+    Label wlCompressionCodec = new Label(shell, SWT.RIGHT);
+    wlCompressionCodec.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.CompressionCodec.Label"));
+    props.setLook(wlCompressionCodec);
+    FormData fdlCompressionCodec = new FormData();
+    fdlCompressionCodec.left = new FormAttachment(0, 0);
+    fdlCompressionCodec.right = new FormAttachment(middle, -margin);
+    fdlCompressionCodec.top = new FormAttachment(lastControl, margin);
+    wlCompressionCodec.setLayoutData(fdlCompressionCodec);
+    wCompressionCodec = new Combo(shell, SWT.SINGLE | SWT.LEFT | SWT.BORDER);
+    for (CompressionCodecName codecName : CompressionCodecName.values()) {
+      wCompressionCodec.add(codecName.name());
+    }
+    props.setLook(wCompressionCodec);
+    FormData fdCompressionCodec = new FormData();
+    fdCompressionCodec.left = new FormAttachment(middle, 0);
+    fdCompressionCodec.top = new FormAttachment(wlCompressionCodec, 0, 
SWT.CENTER);
+    fdCompressionCodec.right = new FormAttachment(100, 0);
+    wCompressionCodec.setLayoutData(fdCompressionCodec);
+    lastControl = wCompressionCodec;
+
+    Label wlVersion = new Label(shell, SWT.RIGHT);
+    wlVersion.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.Version.Label"));
+    props.setLook(wlVersion);
+    FormData fdlVersion = new FormData();
+    fdlVersion.left = new FormAttachment(0, 0);
+    fdlVersion.right = new FormAttachment(middle, -margin);
+    fdlVersion.top = new FormAttachment(lastControl, margin);
+    wlVersion.setLayoutData(fdlVersion);
+    wVersion = new Combo(shell, SWT.SINGLE | SWT.LEFT | SWT.BORDER);
+    for (ParquetVersion version : ParquetVersion.values()) {
+      wVersion.add(version.getDescription());
+    }
+    props.setLook(wVersion);
+    FormData fdVersion = new FormData();
+    fdVersion.left = new FormAttachment(middle, 0);
+    fdVersion.top = new FormAttachment(wlVersion, 0, SWT.CENTER);
+    fdVersion.right = new FormAttachment(100, 0);
+    wVersion.setLayoutData(fdVersion);
+    lastControl = wVersion;
+
+    Label wlRowGroupSize = new Label(shell, SWT.RIGHT);
+    wlRowGroupSize.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.RowGroupSize.Label"));
+    props.setLook(wlRowGroupSize);
+    FormData fdlRowGroupSize = new FormData();
+    fdlRowGroupSize.left = new FormAttachment(0, 0);
+    fdlRowGroupSize.right = new FormAttachment(middle, -margin);
+    fdlRowGroupSize.top = new FormAttachment(lastControl, margin);
+    wlRowGroupSize.setLayoutData(fdlRowGroupSize);
+    wRowGroupSize = new TextVar(variables, shell, SWT.SINGLE | SWT.LEFT | 
SWT.BORDER);
+    props.setLook(wRowGroupSize);
+    FormData fdRowGroupSize = new FormData();
+    fdRowGroupSize.left = new FormAttachment(middle, 0);
+    fdRowGroupSize.top = new FormAttachment(wlRowGroupSize, 0, SWT.CENTER);
+    fdRowGroupSize.right = new FormAttachment(100, 0);
+    wRowGroupSize.setLayoutData(fdRowGroupSize);
+    lastControl = wRowGroupSize;
+
+    Label wlDataPageSize = new Label(shell, SWT.RIGHT);
+    wlDataPageSize.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.DataPageSize.Label"));
+    props.setLook(wlDataPageSize);
+    FormData fdlDataPageSize = new FormData();
+    fdlDataPageSize.left = new FormAttachment(0, 0);
+    fdlDataPageSize.right = new FormAttachment(middle, -margin);
+    fdlDataPageSize.top = new FormAttachment(lastControl, margin);
+    wlDataPageSize.setLayoutData(fdlDataPageSize);
+    wDataPageSize = new TextVar(variables, shell, SWT.SINGLE | SWT.LEFT | 
SWT.BORDER);
+    props.setLook(wDataPageSize);
+    FormData fdDataPageSize = new FormData();
+    fdDataPageSize.left = new FormAttachment(middle, 0);
+    fdDataPageSize.top = new FormAttachment(wlDataPageSize, 0, SWT.CENTER);
+    fdDataPageSize.right = new FormAttachment(100, 0);
+    wDataPageSize.setLayoutData(fdDataPageSize);
+    lastControl = wDataPageSize;
+
+    Label wlDictionaryPageSize = new Label(shell, SWT.RIGHT);
+    wlDictionaryPageSize.setText(
+        BaseMessages.getString(PKG, 
"ParquetOutputDialog.DictionaryPageSize.Label"));
+    props.setLook(wlDictionaryPageSize);
+    FormData fdlDictionaryPageSize = new FormData();
+    fdlDictionaryPageSize.left = new FormAttachment(0, 0);
+    fdlDictionaryPageSize.right = new FormAttachment(middle, -margin);
+    fdlDictionaryPageSize.top = new FormAttachment(lastControl, margin);
+    wlDictionaryPageSize.setLayoutData(fdlDictionaryPageSize);
+    wDictionaryPageSize = new TextVar(variables, shell, SWT.SINGLE | SWT.LEFT 
| SWT.BORDER);
+    props.setLook(wDictionaryPageSize);
+    FormData fdDictionaryPageSize = new FormData();
+    fdDictionaryPageSize.left = new FormAttachment(middle, 0);
+    fdDictionaryPageSize.top = new FormAttachment(wlDictionaryPageSize, 0, 
SWT.CENTER);
+    fdDictionaryPageSize.right = new FormAttachment(100, 0);
+    wDictionaryPageSize.setLayoutData(fdDictionaryPageSize);
+    lastControl = wDictionaryPageSize;
+
+    Label wlFields = new Label(shell, SWT.LEFT);
+    wlFields.setText(BaseMessages.getString(PKG, 
"ParquetOutputDialog.Fields.Label"));
+    props.setLook(wlFields);
+    FormData fdlFields = new FormData();
+    fdlFields.left = new FormAttachment(0, 0);
+    fdlFields.right = new FormAttachment(middle, -margin);
+    fdlFields.top = new FormAttachment(lastControl, margin);
+    wlFields.setLayoutData(fdlFields);
+
+    ColumnInfo[] columns =
+        new ColumnInfo[] {
+          new ColumnInfo(
+              BaseMessages.getString(PKG, 
"ParquetOutputDialog.FieldsColumn.SourceField.Label"),
+              ColumnInfo.COLUMN_TYPE_CCOMBO,
+              new String[0]),
+          new ColumnInfo(
+              BaseMessages.getString(PKG, 
"ParquetOutputDialog.FieldsColumn.TargetField.Label"),
+              ColumnInfo.COLUMN_TYPE_TEXT,
+              false,
+              false),
+        };
+    wFields =
+        new TableView(
+            variables, shell, SWT.NONE, columns, input.getFields().size(), 
false, null, props);
+    props.setLook(wFields);
+    FormData fdFields = new FormData();
+    fdFields.left = new FormAttachment(0, 0);
+    fdFields.top = new FormAttachment(wlFields, margin);
+    fdFields.right = new FormAttachment(100, 0);
+    fdFields.bottom = new FormAttachment(wOk, -2 * margin);
+    wFields.setLayoutData(fdFields);
+
+    getData();
+
+    BaseDialog.defaultShellHandling(shell, c -> ok(), c -> cancel());
+    return returnValue;
+  }
+
+  private void enableFields() {
+    
wlFilenameDateTimeFormat.setEnabled(wFilenameIncludeDateTime.getSelection());
+    
wFilenameDateTimeFormat.setEnabled(wFilenameIncludeDateTime.getSelection());
+
+    wlFilenameSplitSize.setEnabled(wFilenameIncludeSplitNr.getSelection());
+    wFilenameSplitSize.setEnabled(wFilenameIncludeSplitNr.getSelection());
+  }
+
+  private void getFields() {
+    // Populate the wFields grid
+    //
+    try {
+      IRowMeta rowMeta = pipelineMeta.getPrevTransformFields(variables, 
transformName);
+      BaseTransformDialog.getFieldsFromPrevious(
+          rowMeta, wFields, 2, new int[] {1, 2}, new int[0], -1, -1, true, 
null);
+    } catch (Exception e) {
+      new ErrorDialog(shell, "Error", "Error getting fields", e);
+    }
+  }
+
+  private void getData() {
+    try {
+      IRowMeta fields = pipelineMeta.getPrevTransformFields(variables, 
transformName);
+      wFields.getColumns()[0].setComboValues(fields.getFieldNames());
+    } catch (Exception e) {
+      LogChannel.UI.logError("Error getting source fields", e);
+    }
+
+    wTransformName.setText(Const.NVL(transformName, ""));
+    wFilenameBase.setText(Const.NVL(input.getFilenameBase(), ""));
+    wFilenameExtension.setText(Const.NVL(input.getFilenameExtension(), ""));
+    wFilenameIncludeDate.setSelection(input.isFilenameIncludingDate());
+    wFilenameIncludeTime.setSelection(input.isFilenameIncludingTime());
+    wFilenameIncludeDateTime.setSelection(input.isFilenameIncludingDateTime());
+    
wFilenameDateTimeFormat.setText(Const.NVL(input.getFilenameDateTimeFormat(), 
""));
+    wFilenameIncludeCopyNr.setSelection(input.isFilenameIncludingCopyNr());
+    wFilenameIncludeSplitNr.setSelection(input.isFilenameIncludingSplitNr());
+    wFilenameSplitSize.setText(Const.NVL(input.getFileSplitSize(), ""));
+    wCompressionCodec.setText(input.getCompressionCodec().name());
+    wVersion.setText(input.getVersion().getDescription());
+    wRowGroupSize.setText(Const.NVL(input.getRowGroupSize(), ""));
+    wDataPageSize.setText(Const.NVL(input.getDataPageSize(), ""));
+    wDictionaryPageSize.setText(Const.NVL(input.getDictionaryPageSize(), ""));
+    for (int i = 0; i < input.getFields().size(); i++) {
+      ParquetField field = input.getFields().get(i);
+      TableItem item = wFields.table.getItem(i);
+      item.setText(1, Const.NVL(field.getSourceFieldName(), ""));
+      item.setText(2, Const.NVL(field.getTargetFieldName(), ""));
+    }
+  }
+
+  private void ok() {
+    returnValue = wTransformName.getText();
+
+    input.setFilenameBase(wFilenameBase.getText());
+    input.setFilenameExtension(wFilenameExtension.getText());
+    input.setFilenameIncludingDate(wFilenameIncludeDate.getSelection());
+    input.setFilenameIncludingTime(wFilenameIncludeTime.getSelection());
+    
input.setFilenameIncludingDateTime(wFilenameIncludeDateTime.getSelection());
+    input.setFilenameDateTimeFormat(wFilenameDateTimeFormat.getText());
+    input.setFilenameIncludingCopyNr(wFilenameIncludeCopyNr.getSelection());
+    input.setFilenameIncludingSplitNr(wFilenameIncludeSplitNr.getSelection());
+    input.setFileSplitSize(wFilenameSplitSize.getText());
+
+    CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
+    try {
+      codec = CompressionCodecName.valueOf(wCompressionCodec.getText());
+    } catch (Exception e) {
+      // Uncompressed it is.
+    }
+    input.setCompressionCodec(codec);
+
+    
input.setVersion(ParquetVersion.getVersionFromDescription(wVersion.getText()));
+    input.setRowGroupSize(wRowGroupSize.getText());
+    input.setDataPageSize(wDataPageSize.getText());
+    input.setDictionaryPageSize(wDictionaryPageSize.getText());
+    input.getFields().clear();
+    for (TableItem item : wFields.getNonEmptyItems()) {
+      input.getFields().add(new ParquetField(item.getText(1), 
item.getText(2)));
+    }
+    dispose();
+  }
+
+  private void cancel() {
+    returnValue = null;
+    dispose();
+  }
+
+  @Override
+  public void dispose() {
+    props.setScreen(new WindowProperty(shell));
+    shell.dispose();
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputFile.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputFile.java
new file mode 100644
index 0000000..e2aa5a2
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputFile.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.io.PositionOutputStream;
+
+import java.io.OutputStream;
+
+public class ParquetOutputFile implements OutputFile {
+
+  private final OutputStream outputStream;
+
+  public ParquetOutputFile(OutputStream outputStream) {
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public PositionOutputStream create(long blockSizeHint) {
+    return new ParquetOutputStream(outputStream);
+  }
+
+  @Override
+  public PositionOutputStream createOrOverwrite(long blockSizeHint) {
+    return new ParquetOutputStream(outputStream);
+  }
+
+  @Override
+  public boolean supportsBlockSize() {
+    return false;
+  }
+
+  @Override
+  public long defaultBlockSize() {
+    return 0;
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputMeta.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputMeta.java
new file mode 100644
index 0000000..6ef0cb5
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputMeta.java
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hop.core.annotations.Transform;
+import org.apache.hop.metadata.api.HopMetadataProperty;
+import org.apache.hop.pipeline.Pipeline;
+import org.apache.hop.pipeline.PipelineMeta;
+import org.apache.hop.pipeline.transform.BaseTransformMeta;
+import org.apache.hop.pipeline.transform.ITransform;
+import org.apache.hop.pipeline.transform.ITransformMeta;
+import org.apache.hop.pipeline.transform.TransformMeta;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@Transform(
+    id = "ParquetFileOutput",
+    image = "parquet_output.svg",
+    name = "i18n::ParquetOutput.Name",
+    description = "i18n::ParquetOutput.Description",
+    categoryDescription = 
"i18n:org.apache.hop.pipeline.transform:BaseTransform.Category.Output",
+    keywords = {"parquet", "file", "column"})
+public class ParquetOutputMeta extends BaseTransformMeta
+    implements ITransformMeta<ParquetOutput, ParquetOutputData> {
+
+  @HopMetadataProperty(key = "filename_base")
+  private String filenameBase;
+
+  @HopMetadataProperty(key = "filename_ext")
+  private String filenameExtension;
+
+  @HopMetadataProperty(key = "filename_include_date")
+  private boolean filenameIncludingDate;
+
+  @HopMetadataProperty(key = "filename_include_time")
+  private boolean filenameIncludingTime;
+
+  @HopMetadataProperty(key = "filename_include_datetime")
+  private boolean filenameIncludingDateTime;
+
+  @HopMetadataProperty(key = "filename_datetime_format")
+  private String filenameDateTimeFormat;
+
+  @HopMetadataProperty(key = "filename_include_copy")
+  private boolean filenameIncludingCopyNr;
+
+  @HopMetadataProperty(key = "filename_include_split")
+  private boolean filenameIncludingSplitNr;
+
+  @HopMetadataProperty(key = "filename_split_size")
+  private String fileSplitSize;
+
+  @HopMetadataProperty(key = "compression_codec")
+  private CompressionCodecName compressionCodec;
+
+  @HopMetadataProperty(key = "version", storeWithCode = true)
+  private ParquetVersion version;
+
+  @HopMetadataProperty(key = "row_group_size")
+  private String rowGroupSize;
+
+  @HopMetadataProperty(key = "data_page_size")
+  private String dataPageSize;
+
+  @HopMetadataProperty(key = "dictionary_page_size")
+  private String dictionaryPageSize;
+
+  @HopMetadataProperty(groupKey = "fields", key = "field")
+  private List<ParquetField> fields;
+
+  public ParquetOutputMeta() {
+    filenameExtension = "parquet";
+    filenameDateTimeFormat = "yyyyMMdd-HHmmss";
+    compressionCodec = CompressionCodecName.UNCOMPRESSED;
+    version = ParquetVersion.Version1; // The default is v1
+    rowGroupSize = 
Integer.toString(ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT);
+    dataPageSize = Integer.toString(ParquetProperties.DEFAULT_PAGE_SIZE);
+    dictionaryPageSize = 
Integer.toString(ParquetProperties.DEFAULT_DICTIONARY_PAGE_SIZE);
+    fields = new ArrayList<>();
+    filenameIncludingCopyNr = true;
+    filenameIncludingSplitNr = true;
+    fileSplitSize = "1000000";
+  }
+
+  public ParquetOutputMeta(ParquetOutputMeta m) {
+    this.filenameBase = m.filenameBase;
+    this.filenameExtension = m.filenameExtension;
+    this.filenameIncludingDate = m.filenameIncludingDate;
+    this.filenameIncludingTime = m.filenameIncludingTime;
+    this.filenameIncludingDateTime = m.filenameIncludingDateTime;
+    this.filenameDateTimeFormat = m.filenameDateTimeFormat;
+    this.filenameIncludingCopyNr = m.filenameIncludingCopyNr;
+    this.filenameIncludingSplitNr = m.filenameIncludingSplitNr;
+    this.fileSplitSize = m.fileSplitSize;
+    this.compressionCodec = m.compressionCodec;
+    this.version = m.version;
+    this.rowGroupSize = m.rowGroupSize;
+    this.dataPageSize = m.dataPageSize;
+    this.dictionaryPageSize = m.dictionaryPageSize;
+    this.fields = m.fields;
+  }
+
+  @Override
+  public ParquetOutputData getTransformData() {
+    return new ParquetOutputData();
+  }
+
+  @Override
+  public ITransform createTransform(
+      TransformMeta transformMeta,
+      ParquetOutputData data,
+      int copyNr,
+      PipelineMeta pipelineMeta,
+      Pipeline pipeline) {
+    return new ParquetOutput(transformMeta, this, data, copyNr, pipelineMeta, 
pipeline);
+  }
+
+  /**
+   * Gets filenameBase
+   *
+   * @return value of filenameBase
+   */
+  public String getFilenameBase() {
+    return filenameBase;
+  }
+
+  /** @param filenameBase The filenameBase to set */
+  public void setFilenameBase(String filenameBase) {
+    this.filenameBase = filenameBase;
+  }
+
+  /**
+   * Gets filenameExtension
+   *
+   * @return value of filenameExtension
+   */
+  public String getFilenameExtension() {
+    return filenameExtension;
+  }
+
+  /** @param filenameExtension The filenameExtension to set */
+  public void setFilenameExtension(String filenameExtension) {
+    this.filenameExtension = filenameExtension;
+  }
+
+  /**
+   * Gets filenameIncludingDate
+   *
+   * @return value of filenameIncludingDate
+   */
+  public boolean isFilenameIncludingDate() {
+    return filenameIncludingDate;
+  }
+
+  /** @param filenameIncludingDate The filenameIncludingDate to set */
+  public void setFilenameIncludingDate(boolean filenameIncludingDate) {
+    this.filenameIncludingDate = filenameIncludingDate;
+  }
+
+  /**
+   * Gets filenameIncludingTime
+   *
+   * @return value of filenameIncludingTime
+   */
+  public boolean isFilenameIncludingTime() {
+    return filenameIncludingTime;
+  }
+
+  /** @param filenameIncludingTime The filenameIncludingTime to set */
+  public void setFilenameIncludingTime(boolean filenameIncludingTime) {
+    this.filenameIncludingTime = filenameIncludingTime;
+  }
+
+  /**
+   * Gets filenameIncludingDateTime
+   *
+   * @return value of filenameIncludingDateTime
+   */
+  public boolean isFilenameIncludingDateTime() {
+    return filenameIncludingDateTime;
+  }
+
+  /** @param filenameIncludingDateTime The filenameIncludingDateTime to set */
+  public void setFilenameIncludingDateTime(boolean filenameIncludingDateTime) {
+    this.filenameIncludingDateTime = filenameIncludingDateTime;
+  }
+
+  /**
+   * Gets filenameDateTimeFormat
+   *
+   * @return value of filenameDateTimeFormat
+   */
+  public String getFilenameDateTimeFormat() {
+    return filenameDateTimeFormat;
+  }
+
+  /** @param filenameDateTimeFormat The filenameDateTimeFormat to set */
+  public void setFilenameDateTimeFormat(String filenameDateTimeFormat) {
+    this.filenameDateTimeFormat = filenameDateTimeFormat;
+  }
+
+  /**
+   * Gets filenameIncludingCopyNr
+   *
+   * @return value of filenameIncludingCopyNr
+   */
+  public boolean isFilenameIncludingCopyNr() {
+    return filenameIncludingCopyNr;
+  }
+
+  /** @param filenameIncludingCopyNr The filenameIncludingCopyNr to set */
+  public void setFilenameIncludingCopyNr(boolean filenameIncludingCopyNr) {
+    this.filenameIncludingCopyNr = filenameIncludingCopyNr;
+  }
+
+  /**
+   * Gets filenameIncludingSplitNr
+   *
+   * @return value of filenameIncludingSplitNr
+   */
+  public boolean isFilenameIncludingSplitNr() {
+    return filenameIncludingSplitNr;
+  }
+
+  /** @param filenameIncludingSplitNr The filenameIncludingSplitNr to set */
+  public void setFilenameIncludingSplitNr(boolean filenameIncludingSplitNr) {
+    this.filenameIncludingSplitNr = filenameIncludingSplitNr;
+  }
+
+  /**
+   * Gets filenameIncludingSplitSize
+   *
+   * @return value of filenameIncludingSplitSize
+   */
+  public String getFileSplitSize() {
+    return fileSplitSize;
+  }
+
+  /** @param fileSplitSize The filenameIncludingSplitSize to set */
+  public void setFileSplitSize(String fileSplitSize) {
+    this.fileSplitSize = fileSplitSize;
+  }
+
+  /**
+   * Gets compressionCodec
+   *
+   * @return value of compressionCodec
+   */
+  public CompressionCodecName getCompressionCodec() {
+    return compressionCodec;
+  }
+
+  /** @param compressionCodec The compressionCodec to set */
+  public void setCompressionCodec(CompressionCodecName compressionCodec) {
+    this.compressionCodec = compressionCodec;
+  }
+
+  /**
+   * Gets version
+   *
+   * @return value of version
+   */
+  public ParquetVersion getVersion() {
+    return version;
+  }
+
+  /** @param version The version to set */
+  public void setVersion(ParquetVersion version) {
+    this.version = version;
+  }
+
+  /**
+   * Gets rowGroupSize
+   *
+   * @return value of rowGroupSize
+   */
+  public String getRowGroupSize() {
+    return rowGroupSize;
+  }
+
+  /** @param rowGroupSize The rowGroupSize to set */
+  public void setRowGroupSize(String rowGroupSize) {
+    this.rowGroupSize = rowGroupSize;
+  }
+
+  /**
+   * Gets dataPageSize
+   *
+   * @return value of dataPageSize
+   */
+  public String getDataPageSize() {
+    return dataPageSize;
+  }
+
+  /** @param dataPageSize The dataPageSize to set */
+  public void setDataPageSize(String dataPageSize) {
+    this.dataPageSize = dataPageSize;
+  }
+
+  /**
+   * Gets dictionaryPageSize
+   *
+   * @return value of dictionaryPageSize
+   */
+  public String getDictionaryPageSize() {
+    return dictionaryPageSize;
+  }
+
+  /** @param dictionaryPageSize The dictionaryPageSize to set */
+  public void setDictionaryPageSize(String dictionaryPageSize) {
+    this.dictionaryPageSize = dictionaryPageSize;
+  }
+
+  /**
+   * Gets fields
+   *
+   * @return value of fields
+   */
+  public List<ParquetField> getFields() {
+    return fields;
+  }
+
+  /** @param fields The fields to set */
+  public void setFields(List<ParquetField> fields) {
+    this.fields = fields;
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputStream.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputStream.java
new file mode 100644
index 0000000..9e672eb
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetOutputStream.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.parquet.io.PositionOutputStream;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ParquetOutputStream extends PositionOutputStream {
+  private long position = 0;
+  private final OutputStream outputStream;
+
+  public ParquetOutputStream(OutputStream outputStream) {
+    this.outputStream = outputStream;
+  }
+
+  @Override
+  public long getPos() {
+    return position;
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    position++;
+    outputStream.write(b);
+  }
+
+  @Override
+  public void write(byte[] b) throws IOException {
+    write(b, 0, b.length);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    outputStream.write(b, off, len);
+    position += len;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    outputStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    outputStream.close();
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetVersion.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetVersion.java
new file mode 100644
index 0000000..f44a20d
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetVersion.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hop.metadata.api.IEnumHasCode;
+
+public enum ParquetVersion implements IEnumHasCode {
+  Version1("1.0", "Parquet 1.0"),
+  Version2("2.0", "Parquet 2.0");
+
+  private String code;
+  private String description;
+
+  ParquetVersion(String code, String description) {
+    this.code = code;
+    this.description = description;
+  }
+
+  public static final String[] getDescriptions() {
+    String[] descriptions = new String[values().length];
+    for (int i = 0; i < descriptions.length; i++) {
+      descriptions[i] = values()[i].description;
+    }
+    return descriptions;
+  }
+
+  public static ParquetVersion getVersionFromDescription(String description) {
+    for (ParquetVersion version : values()) {
+      if (version.getDescription().equals(description)) {
+        return version;
+      }
+    }
+    return Version1;
+  }
+
+  /**
+   * Gets code
+   *
+   * @return value of code
+   */
+  public String getCode() {
+    return code;
+  }
+
+  /** @param code The code to set */
+  public void setCode(String code) {
+    this.code = code;
+  }
+
+  /**
+   * Gets description
+   *
+   * @return value of description
+   */
+  public String getDescription() {
+    return description;
+  }
+
+  /** @param description The description to set */
+  public void setDescription(String description) {
+    this.description = description;
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriteSupport.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriteSupport.java
new file mode 100644
index 0000000..f7a7993
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriteSupport.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hop.core.RowMetaAndData;
+import org.apache.hop.core.exception.HopException;
+import org.apache.hop.core.row.IValueMeta;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.HashMap;
+import java.util.List;
+
+public class ParquetWriteSupport extends WriteSupport<RowMetaAndData> {
+
+  private final MessageType messageType;
+  private RecordConsumer recordConsumer;
+  private final List<Integer> sourceFieldIndexes;
+  private final List<ParquetField> fields;
+
+  public ParquetWriteSupport(
+      MessageType messageType, List<Integer> sourceFieldIndexes, 
List<ParquetField> fields) {
+    this.messageType = messageType;
+    this.sourceFieldIndexes = sourceFieldIndexes;
+    this.fields = fields;
+  }
+
+  @Override
+  public WriteContext init(Configuration configuration) {
+    return new WriteContext(messageType, new HashMap<>());
+  }
+
+  @Override
+  public void prepareForWrite(RecordConsumer recordConsumer) {
+    this.recordConsumer = recordConsumer;
+  }
+
+  @Override
+  public void write(RowMetaAndData row) {
+    recordConsumer.startMessage();
+    try {
+      // Grab the fields that are mapped...
+      // Write a value
+      //
+      for (int i = 0; i < fields.size(); i++) {
+        ParquetField field = fields.get(i);
+        int index = sourceFieldIndexes.get(i);
+        IValueMeta valueMeta = row.getValueMeta(index);
+        Object valueData = row.getData()[index];
+
+        boolean isNull = valueMeta.isNull(valueData);
+        if (!isNull) {
+          recordConsumer.startField(field.getTargetFieldName(), i);
+          switch (valueMeta.getType()) {
+            case IValueMeta.TYPE_INTEGER:
+              recordConsumer.addLong(valueMeta.getInteger(valueData));
+              break;
+            case IValueMeta.TYPE_NUMBER:
+              recordConsumer.addDouble(valueMeta.getNumber(valueData));
+              break;
+            case IValueMeta.TYPE_BOOLEAN:
+              recordConsumer.addBoolean(valueMeta.getBoolean(valueData));
+              break;
+            case IValueMeta.TYPE_DATE:
+              recordConsumer.addLong(valueMeta.getDate(valueData).getTime());
+              break;
+            case IValueMeta.TYPE_BINARY:
+              byte[] bytes = valueMeta.getBinary(valueData);
+              recordConsumer.addBinary(Binary.fromConstantByteArray(bytes));
+              break;
+            case IValueMeta.TYPE_STRING:
+            default:
+              
recordConsumer.addBinary(Binary.fromString(valueMeta.getString(valueData)));
+              break;
+          }
+        }
+        recordConsumer.endField(field.getTargetFieldName(), i);
+      }
+      recordConsumer.endMessage();
+    } catch (HopException e) {
+      throw new RuntimeException("Error writing row to Parquet", e);
+    }
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriterBuilder.java
 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriterBuilder.java
new file mode 100644
index 0000000..028d287
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/output/ParquetWriterBuilder.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hop.parquet.transforms.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hop.core.RowMetaAndData;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.io.OutputFile;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.List;
+
+public class ParquetWriterBuilder
+    extends ParquetWriter.Builder<RowMetaAndData, ParquetWriterBuilder> {
+
+  private final MessageType messageType;
+  private final List<Integer> sourceFieldIndexes;
+  private final List<ParquetField> fields;
+
+  protected ParquetWriterBuilder(
+      MessageType messageType,
+      OutputFile path,
+      List<Integer> sourceFieldIndexes,
+      List<ParquetField> fields) {
+    super(path);
+    this.messageType = messageType;
+    this.sourceFieldIndexes = sourceFieldIndexes;
+    this.fields = fields;
+  }
+
+  @Override
+  protected ParquetWriterBuilder self() {
+    return this;
+  }
+
+  @Override
+  protected WriteSupport<RowMetaAndData> getWriteSupport(Configuration conf) {
+    return new ParquetWriteSupport(messageType, sourceFieldIndexes, fields);
+  }
+}
diff --git 
a/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/output/messages/messages_en_US.properties
 
b/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/output/messages/messages_en_US.properties
new file mode 100644
index 0000000..61d3331
--- /dev/null
+++ 
b/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/output/messages/messages_en_US.properties
@@ -0,0 +1,37 @@
+#
+# 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.
+#
+ParquetOutput.Name=Parquet File Output 
+ParquetOutput.Description=Writes rows of data to a Parquet file
+ParquetOutputDialog.TransformName.Label=Transform name
+ParquetOutputDialog.FilenameGroup.Label=Filename
+ParquetOutputDialog.FilenameBase.Label=Base file name
+ParquetOutputDialog.FilenameExtension.Label=Extension
+ParquetOutputDialog.FilenameIncludeDate.Label=Include date?
+ParquetOutputDialog.FilenameIncludeTime.Label=Include time?
+ParquetOutputDialog.FilenameIncludeDateTime.Label=Include date-time format?
+ParquetOutputDialog.FilenameDateTimeFormat.Label=Date time format
+ParquetOutputDialog.FilenameIncludeCopyNr.Label=Include transform copy number?
+ParquetOutputDialog.FilenameIncludeSplitNr.Label=Split into parts and include 
number?
+ParquetOutputDialog.FilenameSplitSize.Label=Split size
+ParquetOutputDialog.CompressionCodec.Label=Compression codec
+ParquetOutputDialog.Version.Label=Version
+ParquetOutputDialog.RowGroupSize.Label Row group size
+ParquetOutputDialog.DataPageSize.Label=Data page size
+ParquetOutputDialog.DictionaryPageSize.Label=Dictionary page size
+ParquetOutputDialog.Fields.Label=Fields
+ParquetOutputDialog.FieldsColumn.SourceField.Label=Source field
+ParquetOutputDialog.FieldsColumn.TargetField.Label=Target field
\ No newline at end of file
diff --git a/plugins/tech/parquet/src/main/resources/parquet.svg 
b/plugins/tech/parquet/src/main/resources/parquet.svg
new file mode 100644
index 0000000..dd128ce
--- /dev/null
+++ b/plugins/tech/parquet/src/main/resources/parquet.svg
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<svg
+    xmlns="http://www.w3.org/2000/svg";
+    width="61.965988mm"
+    height="48.596832mm"
+    viewBox="0 0 61.965989 48.596832"
+    version="1.1">
+  <g transform="translate(124.41235,-38.509337)">
+    <path
+        style="fill:#50abf1;fill-opacity:1;stroke-width:0.26458332"
+        d="m -87.639384,48.103807 c -0.83674,-0.517127 -1.51902,-1.079486 
-1.51617,-1.249686 0.005,-0.288357
+       4.23388,-3.121404 4.92767,-3.301058 0.31903,-0.08261 3.16523,1.418646 
3.39527,1.790869 0.0693,0.112184 -0.0795,0.386977
+       -0.33072,0.610652 -0.85926,0.765019 -4.38703,3.103634 -4.67108,3.096539 
-0.15599,-0.0039 -0.96822,-0.430188 -1.80497,-0.947316
+       z m -12.435416,1.582474 c -0.69122,-0.518846 -1.25604,-1.080778 
-1.25516,-1.248736 7.9e-4,-0.167959 2.795806,-1.95109
+       6.210946,-3.962515 l 6.20936,-3.657135 1.59425,0.849728 c 
0.87684,0.46735 1.59425,0.939985 1.59425,1.050299 0,0.313658
+       -12.17573,7.928766 -12.66282,7.919746 -0.23873,-0.004 -0.9996,-0.43254 
-1.690826,-0.951387 z m -22.92651,7.203557
+       -1.41104,-1.42879 0.62633,-0.40588 c 0.34448,-0.22324 7.33677,-4.03737 
15.53841,-8.475859 l 14.912086,-8.069972 1.42593,0.699344
+       c 0.78427,0.38464 1.42594,0.82959 1.42594,0.988779 0,0.217853 
-30.659476,18.121168 -31.032546,18.121168 -0.0407,0 -0.70904,-0.64296
+       -1.48511,-1.42879 z m 39.407826,4.2842 c -1.07349,-0.77011 
-1.99321,-1.52478 -2.04382,-1.67704 -0.0722,-0.21722 9.65735,-8.30344
+       11.86437,-9.860476 0.42906,-0.302698 0.76662,-0.201878 2.60671,0.778566 
1.1626,0.61946 2.112128,1.25218 2.110065,1.40606
+       -0.0053,0.39367 -11.585555,10.58162 -12.134815,10.67581 -0.24789,0.0425 
-1.32902,-0.5528 -2.40251,-1.32292 z m -35.462046,-0.0987
+       c -1.16667,-1.23358 -1.33631,-1.53156 -1.03056,-1.81025 
0.26442,-0.24102 14.93094,-8.9035 16.70498,-9.866459 0.29295,-0.159015
+       3.402316,2.083569 3.30281,2.382099 -0.12863,0.38591 -16.66927,10.77042 
-17.15603,10.77087 -0.23356,2.1e-4 -1.05309,-0.6641
+       -1.8212,-1.47626 z m 20.559606,2.38265 c -0.97752,-0.81138 
-1.772016,-1.61506 -1.765556,-1.78594 0.006,-0.17087 0.869676,-0.91362
+       1.918226,-1.65054 1.04856,-0.73692 5.74979,-4.10043 10.44717,-7.47448 
4.69738,-3.374042 8.73407,-6.134624 8.97041,-6.134624
+       0.63219,0 3.83253,1.850348 3.73038,2.156804 -0.14839,0.44515 
-20.68696,16.35703 -21.11781,16.36063 -0.22302,0.002 -1.20529,-0.66046
+       -2.18282,-1.47185 z m -15.791686,2.41373 c -1.48244,-1.49905 
-1.65932,-1.77866 -1.32291,-2.09124 0.48226,-0.44811
+       24.159556,-15.733344 24.643396,-15.908928 0.37882,-0.137469 
3.65847,1.848334 3.65847,2.215175 0,0.262223 -24.272676,17.260193
+       -24.863536,17.411753 -0.23462,0.0602 -1.12204,-0.62225 
-2.11542,-1.62676 z m 5.13251,5.26842 c -1.03474,-1.15615 -1.68452,-2.08962
+       -1.54513,-2.21974 0.7219,-0.67391 8.17024,-5.8387 8.41597,-5.83577 
0.40037,0.005 3.995396,3.08705 3.995396,3.42553 0,0.41622
+       -8.066246,6.62516 -8.606966,6.62516 -0.26047,0 -1.27714,-0.89783 
-2.25927,-1.99518 z m 6.25912,6.38404 c -1.3218,-1.32871
+       -2.15786,-2.37251 -2.07445,-2.5899 0.15068,-0.39266 17.280786,-14.23912 
17.601056,-14.22713 0.39482,0.0148 4.42634,3.07772
+       4.42634,3.3629 0,0.30012 -17.220596,15.67988 -17.556646,15.67988 
-0.10016,0 -1.17849,-1.00159 -2.3963,-2.22575 z m 6.763806,6.94892
+       c -2.45176,-2.61569 -2.525,-2.72897 -2.08429,-3.22357 0.69857,-0.784 
30.4958,-28.05978 30.923573,-28.30682 0.370096,-0.21373
+       4.023447,1.5502 4.821637,2.328 0.305998,0.29819 -2.068179,2.83579 
-15.02077,16.05471 -8.46132,8.6353 -15.54507,15.73611
+       -15.74165,15.77959 -0.19659,0.0435 -1.50091,-1.14089 -2.8985,-2.63191 
z"/>
+  </g>
+</svg>
diff --git a/plugins/tech/parquet/src/main/resources/parquet_input.svg 
b/plugins/tech/parquet/src/main/resources/parquet_input.svg
new file mode 100644
index 0000000..eb97d81
--- /dev/null
+++ b/plugins/tech/parquet/src/main/resources/parquet_input.svg
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<svg
+    xmlns="http://www.w3.org/2000/svg";
+    width="61.965988mm"
+    height="48.596832mm"
+    viewBox="0 0 61.965989 48.596832"
+    version="1.1">
+  <g transform="matrix(0.97465742,0,0,0.97465742,121.25942,-37.533412)">
+    <path
+        style="fill:#50abf1;fill-opacity:1;stroke-width:0.26458332"
+        d="m -87.639384,48.103807 c -0.83674,-0.517127 -1.51902,-1.079486 
-1.51617,-1.249686 0.005,-0.288357 4.23388,-3.121404
+       4.92767,-3.301058 0.31903,-0.08261 3.16523,1.418646 3.39527,1.790869 
0.0693,0.112184 -0.0795,0.386977 -0.33072,0.610652
+       -0.85926,0.765019 -4.38703,3.103634 -4.67108,3.096539 -0.15599,-0.0039 
-0.96822,-0.430188 -1.80497,-0.947316 z m
+       -12.435416,1.582474 c -0.69122,-0.518846 -1.25604,-1.080778 
-1.25516,-1.248736 7.9e-4,-0.167959 2.795806,-1.95109
+       6.210946,-3.962515 l 6.20936,-3.657135 1.59425,0.849728 c 
0.87684,0.46735 1.59425,0.939985 1.59425,1.050299 0,0.313658
+       -12.17573,7.928766 -12.66282,7.919746 -0.23873,-0.004 -0.9996,-0.43254 
-1.690826,-0.951387 z m -22.92651,7.203557 -1.41104,-1.42879
+       0.62633,-0.40588 c 0.34448,-0.22324 7.33677,-4.03737 15.53841,-8.475859 
l 14.912086,-8.069972 1.42593,0.699344 c 0.78427,0.38464
+       1.42594,0.82959 1.42594,0.988779 0,0.217853 -30.659476,18.121168 
-31.032546,18.121168 -0.0407,0 -0.70904,-0.64296 -1.48511,-1.42879 z
+       m 39.407826,4.2842 c -1.07349,-0.77011 -1.99321,-1.52478 
-2.04382,-1.67704 -0.0722,-0.21722 9.65735,-8.30344 11.86437,-9.860476
+       0.42906,-0.302698 0.76662,-0.201878 2.60671,0.778566 1.1626,0.61946 
2.112128,1.25218 2.110065,1.40606 -0.0053,0.39367 -11.585555,10.58162
+       -12.134815,10.67581 -0.24789,0.0425 -1.32902,-0.5528 -2.40251,-1.32292 
z m -35.462046,-0.0987 c -1.16667,-1.23358 -1.33631,-1.53156
+       -1.03056,-1.81025 0.26442,-0.24102 14.93094,-8.9035 16.70498,-9.866459 
0.29295,-0.159015 3.402316,2.083569 3.30281,2.382099 -0.12863,0.38591
+       -16.66927,10.77042 -17.15603,10.77087 -0.23356,2.1e-4 -1.05309,-0.6641 
-1.8212,-1.47626 z m 20.559606,2.38265 c -0.97752,-0.81138
+       -1.772016,-1.61506 -1.765556,-1.78594 0.006,-0.17087 0.869676,-0.91362 
1.918226,-1.65054 1.04856,-0.73692 5.74979,-4.10043 10.44717,-7.47448
+       4.69738,-3.374042 8.73407,-6.134624 8.97041,-6.134624 0.63219,0 
3.83253,1.850348 3.73038,2.156804 -0.14839,0.44515 -20.68696,16.35703
+       -21.11781,16.36063 -0.22302,0.002 -1.20529,-0.66046 -2.18282,-1.47185 z 
m -15.791686,2.41373 c -1.48244,-1.49905 -1.65932,-1.77866
+       -1.32291,-2.09124 0.48226,-0.44811 24.159556,-15.733344 
24.643396,-15.908928 0.37882,-0.137469 3.65847,1.848334 3.65847,2.215175 
0,0.262223
+       -24.272676,17.260193 -24.863536,17.411753 -0.23462,0.0602 
-1.12204,-0.62225 -2.11542,-1.62676 z m 5.13251,5.26842 c -1.03474,-1.15615
+       -1.68452,-2.08962 -1.54513,-2.21974 0.7219,-0.67391 8.17024,-5.8387 
8.41597,-5.83577 0.40037,0.005 3.995396,3.08705 3.995396,3.42553
+       0,0.41622 -8.066246,6.62516 -8.606966,6.62516 -0.26047,0 
-1.27714,-0.89783 -2.25927,-1.99518 z m 6.25912,6.38404 c -1.3218,-1.32871
+       -2.15786,-2.37251 -2.07445,-2.5899 0.15068,-0.39266 17.280786,-14.23912 
17.601056,-14.22713 0.39482,0.0148 4.42634,3.07772 4.42634,3.3629
+       0,0.30012 -17.220596,15.67988 -17.556646,15.67988 -0.10016,0 
-1.17849,-1.00159 -2.3963,-2.22575 z m 6.763806,6.94892 c -2.45176,-2.61569
+       -2.525,-2.72897 -2.08429,-3.22357 0.69857,-0.784 30.4958,-28.05978 
30.923573,-28.30682 0.370096,-0.21373 4.023447,1.5502 4.821637,2.328
+       0.305998,0.29819 -2.068179,2.83579 -15.02077,16.05471 -8.46132,8.6353 
-15.54507,15.73611 -15.74165,15.77959 -0.19659,0.0435 -1.50091,-1.14089
+       -2.8985,-2.63191 z"/>
+  </g>
+  <polygon
+      transform="matrix(1.6735955,0,0,1.6735955,41.002184,27.503774)"
+      style="fill:#3d6480"
+      points="9.8,8.5 4.6,3.3 3.3,4.7 8.5,9.9 5.3,9.9 5.3,11.8 11.7,11.8 
11.7,5.5 9.8,5.5 "/>
+</svg>
diff --git a/plugins/tech/parquet/src/main/resources/parquet_output.svg 
b/plugins/tech/parquet/src/main/resources/parquet_output.svg
new file mode 100644
index 0000000..b785e88
--- /dev/null
+++ b/plugins/tech/parquet/src/main/resources/parquet_output.svg
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<svg
+    xmlns="http://www.w3.org/2000/svg";
+    width="61.965988mm"
+    height="48.596832mm"
+    viewBox="0 0 61.965989 48.596832"
+    version="1.1">
+  <g transform="matrix(0.8954639,0,0,0.8954639,117.88445,-29.403598)">
+    <path
+        style="fill:#50abf1;fill-opacity:1;stroke-width:0.26458332"
+        d="m -87.639384,48.103807 c -0.83674,-0.517127 -1.51902,-1.079486 
-1.51617,-1.249686 0.005,-0.288357 4.23388,-3.121404 4.92767,-3.301058
+       0.31903,-0.08261 3.16523,1.418646 3.39527,1.790869 0.0693,0.112184 
-0.0795,0.386977 -0.33072,0.610652 -0.85926,0.765019 -4.38703,3.103634
+       -4.67108,3.096539 -0.15599,-0.0039 -0.96822,-0.430188 
-1.80497,-0.947316 z m -12.435416,1.582474 c -0.69122,-0.518846 
-1.25604,-1.080778
+       -1.25516,-1.248736 7.9e-4,-0.167959 2.795806,-1.95109 
6.210946,-3.962515 l 6.20936,-3.657135 1.59425,0.849728 c 0.87684,0.46735
+       1.59425,0.939985 1.59425,1.050299 0,0.313658 -12.17573,7.928766 
-12.66282,7.919746 -0.23873,-0.004 -0.9996,-0.43254 -1.690826,-0.951387 z
+       m -22.92651,7.203557 -1.41104,-1.42879 0.62633,-0.40588 c 
0.34448,-0.22324 7.33677,-4.03737 15.53841,-8.475859 l 14.912086,-8.069972
+       1.42593,0.699344 c 0.78427,0.38464 1.42594,0.82959 1.42594,0.988779 
0,0.217853 -30.659476,18.121168 -31.032546,18.121168 -0.0407,0
+       -0.70904,-0.64296 -1.48511,-1.42879 z m 39.407826,4.2842 c 
-1.07349,-0.77011 -1.99321,-1.52478 -2.04382,-1.67704 -0.0722,-0.21722
+       9.65735,-8.30344 11.86437,-9.860476 0.42906,-0.302698 0.76662,-0.201878 
2.60671,0.778566 1.1626,0.61946 2.112128,1.25218 2.110065,1.40606
+       -0.0053,0.39367 -11.585555,10.58162 -12.134815,10.67581 -0.24789,0.0425 
-1.32902,-0.5528 -2.40251,-1.32292 z m -35.462046,-0.0987 c
+       -1.16667,-1.23358 -1.33631,-1.53156 -1.03056,-1.81025 0.26442,-0.24102 
14.93094,-8.9035 16.70498,-9.866459 0.29295,-0.159015
+       3.402316,2.083569 3.30281,2.382099 -0.12863,0.38591 -16.66927,10.77042 
-17.15603,10.77087 -0.23356,2.1e-4 -1.05309,-0.6641 -1.8212,-1.47626
+       z m 20.559606,2.38265 c -0.97752,-0.81138 -1.772016,-1.61506 
-1.765556,-1.78594 0.006,-0.17087 0.869676,-0.91362 1.918226,-1.65054
+       1.04856,-0.73692 5.74979,-4.10043 10.44717,-7.47448 4.69738,-3.374042 
8.73407,-6.134624 8.97041,-6.134624 0.63219,0 3.83253,1.850348
+       3.73038,2.156804 -0.14839,0.44515 -20.68696,16.35703 -21.11781,16.36063 
-0.22302,0.002 -1.20529,-0.66046 -2.18282,-1.47185 z m
+       -15.791686,2.41373 c -1.48244,-1.49905 -1.65932,-1.77866 
-1.32291,-2.09124 0.48226,-0.44811 24.159556,-15.733344 24.643396,-15.908928
+       0.37882,-0.137469 3.65847,1.848334 3.65847,2.215175 0,0.262223 
-24.272676,17.260193 -24.863536,17.411753 -0.23462,0.0602 -1.12204,-0.62225
+       -2.11542,-1.62676 z m 5.13251,5.26842 c -1.03474,-1.15615 
-1.68452,-2.08962 -1.54513,-2.21974 0.7219,-0.67391 8.17024,-5.8387
+       8.41597,-5.83577 0.40037,0.005 3.995396,3.08705 3.995396,3.42553 
0,0.41622 -8.066246,6.62516 -8.606966,6.62516 -0.26047,0 -1.27714,-0.89783
+       -2.25927,-1.99518 z m 6.25912,6.38404 c -1.3218,-1.32871 
-2.15786,-2.37251 -2.07445,-2.5899 0.15068,-0.39266 17.280786,-14.23912
+       17.601056,-14.22713 0.39482,0.0148 4.42634,3.07772 4.42634,3.3629 
0,0.30012 -17.220596,15.67988 -17.556646,15.67988 -0.10016,0
+       -1.17849,-1.00159 -2.3963,-2.22575 z m 6.763806,6.94892 c 
-2.45176,-2.61569 -2.525,-2.72897 -2.08429,-3.22357 0.69857,-0.784
+       30.4958,-28.05978 30.923573,-28.30682 0.370096,-0.21373 4.023447,1.5502 
4.821637,2.328 0.305998,0.29819 -2.068179,2.83579 -15.02077,16.05471
+       -8.46132,8.6353 -15.54507,15.73611 -15.74165,15.77959 -0.19659,0.0435 
-1.50091,-1.14089 -2.8985,-2.63191 z"/>
+  </g>
+  <polygon
+      transform="matrix(1.6735955,0,0,1.6735955,-2.9446406,-3.9135239)"
+      style="fill:#3d6480"
+      points="3.3,4.7 8.5,9.9 5.3,9.9 5.3,11.8 11.7,11.8 11.7,5.5 9.8,5.5 
9.8,8.5 4.6,3.3 "/>
+</svg>
diff --git a/plugins/tech/pom.xml b/plugins/tech/pom.xml
index 722cdce..9b69b80 100644
--- a/plugins/tech/pom.xml
+++ b/plugins/tech/pom.xml
@@ -57,6 +57,7 @@
         <module>cassandra</module>
         <module>google</module>
         <module>neo4j</module>
+        <module>parquet</module>
       </modules>
     </profile>
 

Reply via email to