This is an automated email from the ASF dual-hosted git repository.
hansva pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/hop.git
The following commit(s) were added to refs/heads/main by this push:
new f1a9681193 Issue #5730 (parquet file input improvements) (#5731)
f1a9681193 is described below
commit f1a96811936a33e8a1eb96693ebe6a6ffb590462
Author: Matt Casters <[email protected]>
AuthorDate: Sun Oct 26 17:27:53 2025 +0100
Issue #5730 (parquet file input improvements) (#5731)
* Issue #5730 (parquet file input improvements)
* Issue #5730 (bug fix to prevent long overflow)
---------
Co-authored-by: Matt Casters <[email protected]>
---
.../pipeline/transforms/parquet-file-input.adoc | 5 +
.../hop/parquet/transforms/input/ParquetInput.java | 32 +++++-
.../transforms/input/ParquetInputDialog.java | 100 ++++-------------
.../parquet/transforms/input/ParquetInputMeta.java | 121 ++++++++++++++++-----
.../transforms/input/ParquetReadSupport.java | 12 +-
.../transforms/input/ParquetValueConverter.java | 29 +++++
.../input/messages/messages_en_US.properties | 1 +
7 files changed, 181 insertions(+), 119 deletions(-)
diff --git
a/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-input.adoc
b/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-input.adoc
index db80b09dfe..16ea061398 100644
---
a/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-input.adoc
+++
b/docs/hop-user-manual/modules/ROOT/pages/pipeline/transforms/parquet-file-input.adoc
@@ -63,6 +63,11 @@ Make sure to allocate enough memory to allow this.
Use a transform like Get File Names to obtain file names.
Any supported file location is fine.
+|Metadata filename
+|If you specify a filename here, you can leave the fields section empty and
Hop will automatically determine
+the output fields. It prevents you from having to define all the fields when
this metadata is already in
+a parquet file schema.
+
|Fields
|In this table you can specify all the fields you want to obtain from the
parquet files as well as their desired Hop output type.
diff --git
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInput.java
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInput.java
index 3629f8f7c9..ab3285bd4b 100644
---
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInput.java
+++
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInput.java
@@ -19,10 +19,14 @@ package org.apache.hop.parquet.transforms.input;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.commons.vfs2.FileObject;
import org.apache.hop.core.RowMetaAndData;
import org.apache.hop.core.exception.HopException;
+import org.apache.hop.core.row.IRowMeta;
+import org.apache.hop.core.row.IValueMeta;
import org.apache.hop.core.row.RowDataUtil;
import org.apache.hop.core.vfs.HopVfs;
import org.apache.hop.pipeline.Pipeline;
@@ -43,7 +47,6 @@ public class ParquetInput extends
BaseTransform<ParquetInputMeta, ParquetInputDa
@Override
public boolean processRow() throws HopException {
-
Object[] row = getRow();
if (row == null) {
// No more files, we're done.
@@ -54,14 +57,15 @@ public class ParquetInput extends
BaseTransform<ParquetInputMeta, ParquetInputDa
if (first) {
first = false;
- data.outputRowMeta = getInputRowMeta().clone();
- meta.getFields(data.outputRowMeta, getTransformName(), null, null, this,
metadataProvider);
data.filenameFieldIndex =
getInputRowMeta().indexOfValue(resolve(meta.getFilenameField()));
if (data.filenameFieldIndex < 0) {
throw new HopException(
"Unable to find filename field " + meta.getFilenameField() + " in
the input");
}
+
+ data.outputRowMeta = getInputRowMeta().clone();
+ meta.getFields(data.outputRowMeta, getTransformName(), null, null, this,
metadataProvider);
}
// Skip null values for file names
@@ -74,6 +78,26 @@ public class ParquetInput extends
BaseTransform<ParquetInputMeta, ParquetInputDa
FileObject fileObject = HopVfs.getFileObject(filename, variables);
try {
+ List<ParquetField> fields = new ArrayList<>(meta.getFields());
+
+ // If we don't have any fields specified, we read them all.
+ //
+ if (fields.isEmpty()) {
+ //
+ IRowMeta parquetRowMeta = ParquetInputMeta.extractRowMeta(this,
filename);
+ for (int i = 0; i < parquetRowMeta.size(); i++) {
+ IValueMeta parquetValueMeta = parquetRowMeta.getValueMeta(i);
+ fields.add(
+ new ParquetField(
+ parquetValueMeta.getName(),
+ parquetValueMeta.getName(),
+ parquetValueMeta.getTypeDesc(),
+ parquetValueMeta.getFormatMask(),
+ Integer.toString(parquetValueMeta.getLength()),
+ Integer.toString(parquetValueMeta.getPrecision())));
+ }
+ }
+
long size = fileObject.getContent().getSize();
data.inputStream = HopVfs.getInputStream(fileObject);
@@ -83,7 +107,7 @@ public class ParquetInput extends
BaseTransform<ParquetInputMeta, ParquetInputDa
IOUtils.copy(data.inputStream, outputStream);
ParquetStream inputFile = new ParquetStream(outputStream.toByteArray(),
filename);
- ParquetReadSupport readSupport = new
ParquetReadSupport(meta.getFields());
+ ParquetReadSupport readSupport = new ParquetReadSupport(fields);
data.reader = new ParquetReaderBuilder<>(readSupport, inputFile).build();
RowMetaAndData r = data.reader.read();
diff --git
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputDialog.java
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputDialog.java
index 243b72349a..98b4b45a49 100644
---
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputDialog.java
+++
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputDialog.java
@@ -17,33 +17,19 @@
package org.apache.hop.parquet.transforms.input;
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.vfs2.FileObject;
import org.apache.hop.core.Const;
-import org.apache.hop.core.RowMetaAndData;
import org.apache.hop.core.logging.LogChannel;
import org.apache.hop.core.row.IRowMeta;
-import org.apache.hop.core.row.IValueMeta;
-import org.apache.hop.core.row.RowMeta;
import org.apache.hop.core.row.value.ValueMetaFactory;
import org.apache.hop.core.variables.IVariables;
-import org.apache.hop.core.vfs.HopVfs;
import org.apache.hop.i18n.BaseMessages;
import org.apache.hop.pipeline.PipelineMeta;
import org.apache.hop.ui.core.PropsUi;
import org.apache.hop.ui.core.dialog.BaseDialog;
-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.column.ColumnDescriptor;
-import org.apache.parquet.hadoop.ParquetReader;
-import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.PrimitiveType;
import org.eclipse.swt.SWT;
import org.eclipse.swt.layout.FormAttachment;
import org.eclipse.swt.layout.FormData;
@@ -63,6 +49,7 @@ public class ParquetInputDialog extends BaseTransformDialog {
protected ParquetInputMeta input;
private Combo wFilenameField;
+ private TextVar wMetaFilename;
private TableView wFields;
private String returnValue;
@@ -143,6 +130,23 @@ public class ParquetInputDialog extends
BaseTransformDialog {
wFilenameField.setLayoutData(fdFilenameField);
lastControl = wFilenameField;
+ Label wlMetaFilename = new Label(shell, SWT.RIGHT);
+ wlMetaFilename.setText(BaseMessages.getString(PKG,
"ParquetInputDialog.MetaFilename.Label"));
+ PropsUi.setLook(wlMetaFilename);
+ FormData fdlMetaFilename = new FormData();
+ fdlMetaFilename.left = new FormAttachment(0, 0);
+ fdlMetaFilename.right = new FormAttachment(middle, -margin);
+ fdlMetaFilename.top = new FormAttachment(lastControl, margin);
+ wlMetaFilename.setLayoutData(fdlMetaFilename);
+ wMetaFilename = new TextVar(variables, shell, SWT.SINGLE | SWT.LEFT |
SWT.BORDER);
+ PropsUi.setLook(wMetaFilename);
+ FormData fdMetaFilename = new FormData();
+ fdMetaFilename.left = new FormAttachment(middle, 0);
+ fdMetaFilename.top = new FormAttachment(wlMetaFilename, 0, SWT.CENTER);
+ fdMetaFilename.right = new FormAttachment(100, 0);
+ wMetaFilename.setLayoutData(fdMetaFilename);
+ lastControl = wMetaFilename;
+
Label wlFields = new Label(shell, SWT.LEFT);
wlFields.setText(BaseMessages.getString(PKG,
"ParquetInputDialog.Fields.Label"));
PropsUi.setLook(wlFields);
@@ -211,63 +215,7 @@ public class ParquetInputDialog extends
BaseTransformDialog {
new String[] {"Parquet files", "All files"},
true);
if (filename != null) {
- FileObject fileObject =
HopVfs.getFileObject(variables.resolve(filename), variables);
-
- long size = fileObject.getContent().getSize();
- InputStream inputStream = HopVfs.getInputStream(fileObject);
-
- // Reads the whole file into memory...
- //
- ByteArrayOutputStream outputStream = new ByteArrayOutputStream((int)
size);
- IOUtils.copy(inputStream, outputStream);
- ParquetStream inputFile = new
ParquetStream(outputStream.toByteArray(), filename);
- // Empty list of fields to retrieve: we still grab the schema
- //
- ParquetReadSupport readSupport = new ParquetReadSupport(new
ArrayList<>());
- ParquetReader<RowMetaAndData> reader =
- new ParquetReaderBuilder<>(readSupport, inputFile).build();
-
- // Read one empty row...
- //
- reader.read();
-
- // Now we have the schema...
- //
- MessageType schema = readSupport.getMessageType();
- IRowMeta rowMeta = new RowMeta();
- List<ColumnDescriptor> columns = schema.getColumns();
- for (ColumnDescriptor column : columns) {
- String sourceField = "";
- String[] path = column.getPath();
- if (path.length == 1) {
- sourceField = path[0];
- } else {
- for (int i = 0; i < path.length; i++) {
- if (i > 0) {
- sourceField += ".";
- }
- sourceField += path[i];
- }
- }
- PrimitiveType primitiveType = column.getPrimitiveType();
- int hopType = IValueMeta.TYPE_STRING;
- switch (primitiveType.getPrimitiveTypeName()) {
- case INT32, INT64:
- hopType = IValueMeta.TYPE_INTEGER;
- break;
- case INT96:
- hopType = IValueMeta.TYPE_BINARY;
- break;
- case FLOAT, DOUBLE:
- hopType = IValueMeta.TYPE_NUMBER;
- break;
- case BOOLEAN:
- hopType = IValueMeta.TYPE_BOOLEAN;
- break;
- }
- IValueMeta valueMeta = ValueMetaFactory.createValueMeta(sourceField,
hopType, -1, -1);
- rowMeta.addValueMeta(valueMeta);
- }
+ IRowMeta rowMeta = ParquetInputMeta.extractRowMeta(variables,
filename);
BaseTransformDialog.getFieldsFromPrevious(
rowMeta, wFields, 1, new int[] {1, 2}, new int[] {3}, -1, -1,
null);
@@ -287,6 +235,7 @@ public class ParquetInputDialog extends BaseTransformDialog
{
wTransformName.setText(Const.NVL(transformName, ""));
wFilenameField.setText(Const.NVL(input.getFilenameField(), ""));
+ wMetaFilename.setText(Const.NVL(input.getMetadataFilename(), ""));
for (int i = 0; i < input.getFields().size(); i++) {
ParquetField field = input.getFields().get(i);
TableItem item = wFields.table.getItem(i);
@@ -310,6 +259,7 @@ public class ParquetInputDialog extends BaseTransformDialog
{
private void getInfo(ParquetInputMeta meta) {
meta.setFilenameField(wFilenameField.getText());
+ meta.setMetadataFilename(wMetaFilename.getText());
meta.getFields().clear();
for (TableItem item : wFields.getNonEmptyItems()) {
int index = 1;
@@ -329,10 +279,4 @@ public class ParquetInputDialog extends
BaseTransformDialog {
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/input/ParquetInputMeta.java
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputMeta.java
index 1d2f1c092e..43c1086e11 100644
---
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputMeta.java
+++
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetInputMeta.java
@@ -17,19 +17,36 @@
package org.apache.hop.parquet.transforms.input;
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
import java.util.ArrayList;
import java.util.List;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.vfs2.FileObject;
+import org.apache.hop.core.RowMetaAndData;
import org.apache.hop.core.annotations.Transform;
import org.apache.hop.core.exception.HopException;
import org.apache.hop.core.exception.HopTransformException;
import org.apache.hop.core.row.IRowMeta;
import org.apache.hop.core.row.IValueMeta;
+import org.apache.hop.core.row.RowMeta;
+import org.apache.hop.core.row.value.ValueMetaFactory;
import org.apache.hop.core.variables.IVariables;
+import org.apache.hop.core.vfs.HopVfs;
import org.apache.hop.metadata.api.HopMetadataProperty;
import org.apache.hop.metadata.api.IHopMetadataProvider;
import org.apache.hop.pipeline.transform.BaseTransformMeta;
import org.apache.hop.pipeline.transform.TransformMeta;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+@Getter
+@Setter
@Transform(
id = "ParquetFileInput",
image = "parquet_input.svg",
@@ -43,6 +60,9 @@ public class ParquetInputMeta extends
BaseTransformMeta<ParquetInput, ParquetInp
@HopMetadataProperty(key = "filename_field")
private String filenameField;
+ @HopMetadataProperty(key = "metadata_filename")
+ private String metadataFilename;
+
@HopMetadataProperty(groupKey = "fields", key = "field")
private List<ParquetField> fields;
@@ -59,6 +79,19 @@ public class ParquetInputMeta extends
BaseTransformMeta<ParquetInput, ParquetInp
IVariables variables,
IHopMetadataProvider metadataProvider)
throws HopTransformException {
+
+ // If there is a filename from which to extra the field metadata, use this
+ //
+ if (fields.isEmpty() && StringUtils.isNotEmpty(metadataFilename)) {
+ String filename = variables.resolve(metadataFilename);
+ try {
+ inputRowMeta.addRowMeta(extractRowMeta(variables, filename));
+ return;
+ } catch (Exception e) {
+ throw new HopTransformException(e);
+ }
+ }
+
// Add the fields to the input
//
for (ParquetField field : fields) {
@@ -73,35 +106,69 @@ public class ParquetInputMeta extends
BaseTransformMeta<ParquetInput, ParquetInp
}
}
- /**
- * Gets filenameField
- *
- * @return value of filenameField
- */
- public String getFilenameField() {
- return filenameField;
- }
+ public static IRowMeta extractRowMeta(IVariables variables, String filename)
throws HopException {
+ try {
+ FileObject fileObject =
HopVfs.getFileObject(variables.resolve(filename), variables);
- /**
- * @param filenameField The filenameField to set
- */
- public void setFilenameField(String filenameField) {
- this.filenameField = filenameField;
- }
+ long size = fileObject.getContent().getSize();
+ InputStream inputStream = HopVfs.getInputStream(fileObject);
- /**
- * Gets fields
- *
- * @return value of fields
- */
- public List<ParquetField> getFields() {
- return fields;
- }
+ // Reads the whole file into memory...
+ //
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream((int)
size);
+ IOUtils.copy(inputStream, outputStream);
+ ParquetStream inputFile = new ParquetStream(outputStream.toByteArray(),
filename);
+ // Empty list of fields to retrieve: we still grab the schema
+ //
+ ParquetReadSupport readSupport = new ParquetReadSupport(new
ArrayList<>());
+ ParquetReader<RowMetaAndData> reader =
+ new ParquetReaderBuilder<>(readSupport, inputFile).build();
- /**
- * @param fields The fields to set
- */
- public void setFields(List<ParquetField> fields) {
- this.fields = fields;
+ // Read one empty row...
+ //
+ reader.read();
+
+ // Now we have the schema...
+ //
+ MessageType schema = readSupport.getMessageType();
+ IRowMeta rowMeta = new RowMeta();
+ List<ColumnDescriptor> columns = schema.getColumns();
+ for (ColumnDescriptor column : columns) {
+ String sourceField = "";
+ String[] path = column.getPath();
+ if (path.length == 1) {
+ sourceField = path[0];
+ } else {
+ for (int i = 0; i < path.length; i++) {
+ if (i > 0) {
+ sourceField += ".";
+ }
+ sourceField += path[i];
+ }
+ }
+ PrimitiveType primitiveType = column.getPrimitiveType();
+ int hopType = IValueMeta.TYPE_STRING;
+ switch (primitiveType.getPrimitiveTypeName()) {
+ case INT32, INT64:
+ hopType = IValueMeta.TYPE_INTEGER;
+ break;
+ case INT96:
+ hopType = IValueMeta.TYPE_TIMESTAMP;
+ break;
+ case FLOAT, DOUBLE:
+ hopType = IValueMeta.TYPE_NUMBER;
+ break;
+ case BOOLEAN:
+ hopType = IValueMeta.TYPE_BOOLEAN;
+ break;
+ }
+ IValueMeta valueMeta = ValueMetaFactory.createValueMeta(sourceField,
hopType, -1, -1);
+ rowMeta.addValueMeta(valueMeta);
+ }
+ return rowMeta;
+ } catch (Exception e) {
+ throw new HopException(
+ "Unable to extract row metadata from parquet file '" + filename +
"'", e);
+ }
}
}
diff --git
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetReadSupport.java
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetReadSupport.java
index 30374903eb..b2364bed63 100644
---
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetReadSupport.java
+++
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetReadSupport.java
@@ -20,6 +20,7 @@ package org.apache.hop.parquet.transforms.input;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import lombok.Getter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hop.core.RowMetaAndData;
import org.apache.parquet.hadoop.api.InitContext;
@@ -35,7 +36,7 @@ public class ParquetReadSupport extends
ReadSupport<RowMetaAndData> {
this.fields = fields;
}
- private MessageType messageType;
+ @Getter private MessageType messageType;
@Override
public ReadContext init(InitContext context) {
@@ -51,13 +52,4 @@ public class ParquetReadSupport extends
ReadSupport<RowMetaAndData> {
ReadContext readContext) {
return new ParquetRecordMaterializer(messageType, fields);
}
-
- /**
- * Gets messageType
- *
- * @return value of messageType
- */
- public MessageType getMessageType() {
- return messageType;
- }
}
diff --git
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetValueConverter.java
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetValueConverter.java
index 72e3d9fbda..5611dba77c 100644
---
a/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetValueConverter.java
+++
b/plugins/tech/parquet/src/main/java/org/apache/hop/parquet/transforms/input/ParquetValueConverter.java
@@ -20,6 +20,8 @@ package org.apache.hop.parquet.transforms.input;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.sql.Timestamp;
import java.util.Date;
import org.apache.hop.core.RowMetaAndData;
import org.apache.hop.core.row.IValueMeta;
@@ -58,6 +60,33 @@ public class ParquetValueConverter extends
PrimitiveConverter {
object = binaryToDecimal(value, valueMeta.getLength(),
valueMeta.getPrecision());
}
break;
+ case IValueMeta.TYPE_TIMESTAMP:
+ if (value.length() == 12) {
+ // This is a binary form of an int96 (12-byte) Timestamp with
nanosecond precision.
+ // The first 8 bytes are the nanoseconds in a day.
+ // The next 4 bytes are the Julian day.
+ // Note: Little Endian.
+ //
+ ByteBuffer bb =
ByteBuffer.wrap(value.getBytes()).order(ByteOrder.LITTLE_ENDIAN);
+ long nsDay = bb.getLong();
+ long julianDay = bb.getInt() & 0x00000000ffffffffL;
+
+ // We need a big integer to prevent a long overflow resulting in
negative values
+ // for: nanoseconds since 1970/01/01 00:00:00
+ //
+ BigInteger bns =
+ BigInteger.valueOf(julianDay - 2440588L)
+ .multiply(BigInteger.valueOf(86400L * 1000 * 1000 * 1000))
+ .add(BigInteger.valueOf(nsDay));
+ BigInteger bms = bns.divide(BigInteger.valueOf(1000000));
+ long ms = bms.longValue();
+ int nanos = (int) (ms % 1000000000);
+ Timestamp timestamp = new Timestamp(ms);
+ timestamp.setNanos(nanos);
+ object = timestamp;
+ break;
+ }
+
default:
throw new RuntimeException(
"Unable to convert Binary source data to type " +
valueMeta.getTypeDesc());
diff --git
a/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/input/messages/messages_en_US.properties
b/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/input/messages/messages_en_US.properties
index 5729209a62..2189f8bac4 100644
---
a/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/input/messages/messages_en_US.properties
+++
b/plugins/tech/parquet/src/main/resources/org/apache/hop/parquet/transforms/input/messages/messages_en_US.properties
@@ -28,3 +28,4 @@ ParquetInputDialog.FieldsColumn.TargetType.Label=Type
ParquetInputDialog.FilenameField.Label=Filename field
ParquetInputDialog.TransformName.Label=Transform name
ParquetInputMeta.keyword=Parquet,input
+ParquetInputDialog.MetaFilename.Label = Metadata filename
\ No newline at end of file