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 d65e68f HOP-2951 : Parquet File Output: add option to create parent
folders
new 1cd7b63 Merge pull request #874 from mattcasters/master
d65e68f is described below
commit d65e68f8ce96b0ef5eb64d4ebefb072eb9949941
Author: Matt Casters <[email protected]>
AuthorDate: Wed Jun 16 09:27:01 2021 +0200
HOP-2951 : Parquet File Output: add option to create parent folders
---
.../transforms/0029-parquet-input.hpl | 6 +-
.../transforms/0029-parquet-output.hpl | 149 +++++++++++----------
.../parquet/transforms/output/ParquetOutput.java | 14 ++
.../transforms/output/ParquetOutputDialog.java | 23 ++++
.../transforms/output/ParquetOutputMeta.java | 19 +++
.../output/messages/messages_en_US.properties | 1 +
6 files changed, 135 insertions(+), 77 deletions(-)
diff --git a/integration-tests/transforms/0029-parquet-input.hpl
b/integration-tests/transforms/0029-parquet-input.hpl
index eb283d5..7d288da 100644
--- a/integration-tests/transforms/0029-parquet-input.hpl
+++ b/integration-tests/transforms/0029-parquet-input.hpl
@@ -41,7 +41,7 @@ limitations under the License.
</notepads>
<order>
<hop>
- <from>${java.io.tmpdir}/customers.parquet.snappy</from>
+ <from>${java.io.tmpdir}/it/parquet/customers.parquet.snappy</from>
<to>only filename</to>
<enabled>Y</enabled>
</hop>
@@ -57,7 +57,7 @@ limitations under the License.
</hop>
</order>
<transform>
- <name>${java.io.tmpdir}/customers.parquet.snappy</name>
+ <name>${java.io.tmpdir}/it/parquet/customers.parquet.snappy</name>
<type>GetFileNames</type>
<description/>
<distribute>Y</distribute>
@@ -81,7 +81,7 @@ limitations under the License.
<dynamic_include_subfolders>N</dynamic_include_subfolders>
<limit>0</limit>
<file>
- <name>${java.io.tmpdir}/customers.parquet.snappy</name>
+ <name>${java.io.tmpdir}/it/parquet/customers.parquet.snappy</name>
<filemask/>
<exclude_filemask/>
<file_required>N</file_required>
diff --git a/integration-tests/transforms/0029-parquet-output.hpl
b/integration-tests/transforms/0029-parquet-output.hpl
index c36f2a4..4fabba2 100644
--- a/integration-tests/transforms/0029-parquet-output.hpl
+++ b/integration-tests/transforms/0029-parquet-output.hpl
@@ -34,7 +34,7 @@ limitations under the License.
<created_date>2021/06/10 15:12:24.183</created_date>
<modified_user>-</modified_user>
<modified_date>2021/06/10 15:12:24.183</modified_date>
- <key_for_session_key/>
+ <key_for_session_key>H4sIAAAAAAAAAAMAAAAAAAAAAAA=</key_for_session_key>
<is_key_private>N</is_key_private>
</info>
<notepads>
@@ -47,6 +47,80 @@ limitations under the License.
</hop>
</order>
<transform>
+ <name>${java.io.tmpdir}/customers.parquet.snappy</name>
+ <type>ParquetFileOutput</type>
+ <description/>
+ <distribute>Y</distribute>
+ <custom_distribution/>
+ <copies>1</copies>
+ <partitioning>
+ <method>none</method>
+ <schema_name/>
+ </partitioning>
+ <compression_codec>SNAPPY</compression_codec>
+ <data_page_size>1048576</data_page_size>
+ <dictionary_page_size>1048576</dictionary_page_size>
+ <fields>
+ <field>
+ <source_field>id</source_field>
+ <target_field>id</target_field>
+ </field>
+ <field>
+ <source_field>name</source_field>
+ <target_field>name</target_field>
+ </field>
+ <field>
+ <source_field>firstname</source_field>
+ <target_field>firstname</target_field>
+ </field>
+ <field>
+ <source_field>zip</source_field>
+ <target_field>zip</target_field>
+ </field>
+ <field>
+ <source_field>city</source_field>
+ <target_field>city</target_field>
+ </field>
+ <field>
+ <source_field>birthdate</source_field>
+ <target_field>birthdate</target_field>
+ </field>
+ <field>
+ <source_field>street</source_field>
+ <target_field>street</target_field>
+ </field>
+ <field>
+ <source_field>housenr</source_field>
+ <target_field>housenr</target_field>
+ </field>
+ <field>
+ <source_field>stateCode</source_field>
+ <target_field>stateCode</target_field>
+ </field>
+ <field>
+ <source_field>state</source_field>
+ <target_field>state</target_field>
+ </field>
+ </fields>
+ <filename_split_size>1000000</filename_split_size>
+ <filename_base>${java.io.tmpdir}/it/parquet/customers</filename_base>
+ <filename_create_parent_folders>Y</filename_create_parent_folders>
+ <filename_datetime_format>yyyyMMdd-HHmmss</filename_datetime_format>
+ <filename_ext>parquet</filename_ext>
+ <filename_include_copy>N</filename_include_copy>
+ <filename_include_date>N</filename_include_date>
+ <filename_include_datetime>N</filename_include_datetime>
+ <filename_include_split>N</filename_include_split>
+ <filename_include_time>N</filename_include_time>
+ <row_group_size>20000</row_group_size>
+ <version>1.0</version>
+ <attributes/>
+ <GUI>
+ <xloc>384</xloc>
+ <yloc>144</yloc>
+ </GUI>
+ </transform>
+ <transform>
<name>customers-100.txt</name>
<type>CSVInput</type>
<description/>
@@ -188,79 +262,6 @@ limitations under the License.
<yloc>144</yloc>
</GUI>
</transform>
- <transform>
- <name>${java.io.tmpdir}/customers.parquet.snappy</name>
- <type>ParquetFileOutput</type>
- <description/>
- <distribute>Y</distribute>
- <custom_distribution/>
- <copies>1</copies>
- <partitioning>
- <method>none</method>
- <schema_name/>
- </partitioning>
- <compression_codec>SNAPPY</compression_codec>
- <data_page_size>1048576</data_page_size>
- <dictionary_page_size>1048576</dictionary_page_size>
- <fields>
- <field>
- <source_field>id</source_field>
- <target_field>id</target_field>
- </field>
- <field>
- <source_field>name</source_field>
- <target_field>name</target_field>
- </field>
- <field>
- <source_field>firstname</source_field>
- <target_field>firstname</target_field>
- </field>
- <field>
- <source_field>zip</source_field>
- <target_field>zip</target_field>
- </field>
- <field>
- <source_field>city</source_field>
- <target_field>city</target_field>
- </field>
- <field>
- <source_field>birthdate</source_field>
- <target_field>birthdate</target_field>
- </field>
- <field>
- <source_field>street</source_field>
- <target_field>street</target_field>
- </field>
- <field>
- <source_field>housenr</source_field>
- <target_field>housenr</target_field>
- </field>
- <field>
- <source_field>stateCode</source_field>
- <target_field>stateCode</target_field>
- </field>
- <field>
- <source_field>state</source_field>
- <target_field>state</target_field>
- </field>
- </fields>
- <filename_split_size>1000000</filename_split_size>
- <filename_base>${java.io.tmpdir}/customers</filename_base>
- <filename_datetime_format>yyyyMMdd-HHmmss</filename_datetime_format>
- <filename_ext>parquet</filename_ext>
- <filename_include_copy>N</filename_include_copy>
- <filename_include_date>N</filename_include_date>
- <filename_include_datetime>N</filename_include_datetime>
- <filename_include_split>N</filename_include_split>
- <filename_include_time>N</filename_include_time>
- <row_group_size>20000</row_group_size>
- <version>1.0</version>
- <attributes/>
- <GUI>
- <xloc>384</xloc>
- <yloc>144</yloc>
- </GUI>
- </transform>
<transform_error_handling>
</transform_error_handling>
<attributes/>
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
index 0d3cc91..f9c9174 100644
---
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
@@ -17,6 +17,7 @@
package org.apache.hop.parquet.transforms.output;
+import org.apache.commons.vfs2.FileObject;
import org.apache.hadoop.conf.Configuration;
import org.apache.hop.core.Const;
import org.apache.hop.core.RowMetaAndData;
@@ -178,6 +179,19 @@ public class ParquetOutput extends
BaseTransform<ParquetOutputMeta, ParquetOutpu
data.filename = buildFilename(getPipeline().getExecutionStartDate());
try {
+ FileObject fileObject = HopVfs.getFileObject(data.filename);
+
+ // See if we need to create the parent folder(s)...
+ //
+ if (meta.isFilenameCreatingParentFolders()) {
+ FileObject parentFolder = fileObject.getParent();
+ if (parentFolder != null && !parentFolder.exists()) {
+ // Try to create the parent folder...
+ //
+ parentFolder.createFolder();
+ }
+ }
+
data.outputStream = HopVfs.getOutputStream(data.filename, false);
data.outputFile = new ParquetOutputFile(data.outputStream);
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
index 32e2833..8a34efd 100644
---
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
@@ -64,6 +64,7 @@ public class ParquetOutputDialog extends BaseTransformDialog
implements ITransfo
private Button wFilenameIncludeSplitNr;
private Label wlFilenameSplitSize;
private TextVar wFilenameSplitSize;
+ private Button wFilenameCreateFolders;
private Combo wCompressionCodec;
private Combo wVersion;
private TextVar wRowGroupSize;
@@ -304,6 +305,24 @@ public class ParquetOutputDialog extends
BaseTransformDialog implements ITransfo
fdFilenameSplitSize.top = new FormAttachment(wlFilenameSplitSize, 0,
SWT.CENTER);
fdFilenameSplitSize.right = new FormAttachment(100, 0);
wFilenameSplitSize.setLayoutData(fdFilenameSplitSize);
+ lastControl = wFilenameSplitSize;
+
+ Label wlFilenameCreateFolders = new Label(wFileGroup, SWT.RIGHT);
+ wlFilenameCreateFolders.setText(
+ BaseMessages.getString(PKG,
"ParquetOutputDialog.FilenameCreateFolders.Label"));
+ props.setLook(wlFilenameCreateFolders);
+ FormData fdlFilenameCreateFolders = new FormData();
+ fdlFilenameCreateFolders.left = new FormAttachment(0, 0);
+ fdlFilenameCreateFolders.right = new FormAttachment(middle, -margin);
+ fdlFilenameCreateFolders.top = new FormAttachment(lastControl, margin);
+ wlFilenameCreateFolders.setLayoutData(fdlFilenameCreateFolders);
+ wFilenameCreateFolders = new Button(wFileGroup, SWT.CHECK);
+ props.setLook(wFilenameCreateFolders);
+ FormData fdFilenameCreateFolders = new FormData();
+ fdFilenameCreateFolders.left = new FormAttachment(middle, 0);
+ fdFilenameCreateFolders.top = new FormAttachment(wlFilenameCreateFolders,
0, SWT.CENTER);
+ fdFilenameCreateFolders.right = new FormAttachment(100, 0);
+ wFilenameCreateFolders.setLayoutData(fdFilenameCreateFolders);
// End of the file group
//
@@ -478,6 +497,7 @@ public class ParquetOutputDialog extends
BaseTransformDialog implements ITransfo
wFilenameIncludeCopyNr.setSelection(input.isFilenameIncludingCopyNr());
wFilenameIncludeSplitNr.setSelection(input.isFilenameIncludingSplitNr());
wFilenameSplitSize.setText(Const.NVL(input.getFileSplitSize(), ""));
+
wFilenameCreateFolders.setSelection(input.isFilenameCreatingParentFolders());
wCompressionCodec.setText(input.getCompressionCodec().name());
wVersion.setText(input.getVersion().getDescription());
wRowGroupSize.setText(Const.NVL(input.getRowGroupSize(), ""));
@@ -489,6 +509,8 @@ public class ParquetOutputDialog extends
BaseTransformDialog implements ITransfo
item.setText(1, Const.NVL(field.getSourceFieldName(), ""));
item.setText(2, Const.NVL(field.getTargetFieldName(), ""));
}
+ wFields.optimizeTableView();
+ enableFields();
}
private void ok() {
@@ -503,6 +525,7 @@ public class ParquetOutputDialog extends
BaseTransformDialog implements ITransfo
input.setFilenameIncludingCopyNr(wFilenameIncludeCopyNr.getSelection());
input.setFilenameIncludingSplitNr(wFilenameIncludeSplitNr.getSelection());
input.setFileSplitSize(wFilenameSplitSize.getText());
+
input.setFilenameCreatingParentFolders(wFilenameCreateFolders.getSelection());
CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
try {
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
index 8b686ac..ac5ce52 100644
---
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
@@ -70,6 +70,9 @@ public class ParquetOutputMeta extends BaseTransformMeta
@HopMetadataProperty(key = "filename_split_size")
private String fileSplitSize;
+ @HopMetadataProperty(key = "filename_create_parent_folders")
+ private boolean filenameCreatingParentFolders;
+
@HopMetadataProperty(key = "compression_codec")
private CompressionCodecName compressionCodec;
@@ -99,6 +102,7 @@ public class ParquetOutputMeta extends BaseTransformMeta
fields = new ArrayList<>();
filenameIncludingCopyNr = true;
filenameIncludingSplitNr = true;
+ filenameCreatingParentFolders = true;
fileSplitSize = "1000000";
}
@@ -112,6 +116,7 @@ public class ParquetOutputMeta extends BaseTransformMeta
this.filenameIncludingCopyNr = m.filenameIncludingCopyNr;
this.filenameIncludingSplitNr = m.filenameIncludingSplitNr;
this.fileSplitSize = m.fileSplitSize;
+ this.filenameCreatingParentFolders = m.filenameCreatingParentFolders;
this.compressionCodec = m.compressionCodec;
this.version = m.version;
this.rowGroupSize = m.rowGroupSize;
@@ -262,6 +267,20 @@ public class ParquetOutputMeta extends BaseTransformMeta
}
/**
+ * Gets filenameCreatingParentFolders
+ *
+ * @return value of filenameCreatingParentFolders
+ */
+ public boolean isFilenameCreatingParentFolders() {
+ return filenameCreatingParentFolders;
+ }
+
+ /** @param filenameCreatingParentFolders The filenameCreatingParentFolders
to set */
+ public void setFilenameCreatingParentFolders(boolean
filenameCreatingParentFolders) {
+ this.filenameCreatingParentFolders = filenameCreatingParentFolders;
+ }
+
+ /**
* Gets compressionCodec
*
* @return value of compressionCodec
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
index 61d3331..7b7dc6d 100644
---
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
@@ -27,6 +27,7 @@ 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.FilenameCreateFolders.Label=Create parent folders?
ParquetOutputDialog.CompressionCodec.Label=Compression codec
ParquetOutputDialog.Version.Label=Version
ParquetOutputDialog.RowGroupSize.Label Row group size