Skip to content

Commit 94fa794

Browse files
committed
NIFI-14538 Added support for HSSF format in the SplitExcel processor
1 parent 8c876da commit 94fa794

File tree

2 files changed

+140
-51
lines changed
  • nifi-extension-bundles/nifi-poi-bundle/nifi-poi-services/src

2 files changed

+140
-51
lines changed

nifi-extension-bundles/nifi-poi-bundle/nifi-poi-services/src/main/java/org/apache/nifi/processors/excel/SplitExcel.java

Lines changed: 126 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import org.apache.nifi.annotation.documentation.CapabilityDescription;
2828
import org.apache.nifi.annotation.documentation.Tags;
2929
import org.apache.nifi.components.PropertyDescriptor;
30+
import org.apache.nifi.excel.InputFileType;
3031
import org.apache.nifi.excel.ProtectionType;
3132
import org.apache.nifi.flowfile.FlowFile;
3233
import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -36,6 +37,10 @@
3637
import org.apache.nifi.processor.Relationship;
3738
import org.apache.nifi.processor.exception.ProcessException;
3839
import org.apache.nifi.processor.util.StandardValidators;
40+
import org.apache.poi.hssf.record.crypto.Biff8EncryptionKey;
41+
import org.apache.poi.hssf.usermodel.HSSFRow;
42+
import org.apache.poi.hssf.usermodel.HSSFSheet;
43+
import org.apache.poi.hssf.usermodel.HSSFWorkbook;
3944
import org.apache.poi.ss.usermodel.Cell;
4045
import org.apache.poi.ss.usermodel.CellCopyContext;
4146
import org.apache.poi.ss.usermodel.CellCopyPolicy;
@@ -48,9 +53,12 @@
4853
import org.apache.poi.xssf.streaming.SXSSFSheet;
4954
import org.apache.poi.xssf.streaming.SXSSFWorkbook;
5055

56+
import java.io.IOException;
57+
import java.io.InputStream;
5158
import java.io.OutputStream;
5259
import java.util.ArrayList;
5360
import java.util.HashMap;
61+
import java.util.Iterator;
5462
import java.util.List;
5563
import java.util.Map;
5664
import java.util.Set;
@@ -66,10 +74,10 @@
6674
@Tags({"split", "text"})
6775
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
6876
@CapabilityDescription("This processor splits a multi sheet Microsoft Excel spreadsheet into multiple Microsoft Excel spreadsheets where each sheet from the original" +
69-
" file is converted to an individual spreadsheet in its own flow file. Currently this processor is only capable of processing .xlsx" +
70-
" (XSSF 2007 OOXML file format) Excel documents and not older .xls (HSSF '97(-2007) file format) documents." +
71-
" Please note all original cell styles are dropped and formulas are removed leaving only the calculated values." +
72-
" Even a single sheet Microsoft Excel spreadsheet is converted to its own flow file with all the original cell styles dropped and formulas removed."
77+
" file is converted to an individual spreadsheet in its own flow file. This processor is capable of processing both password and non password protected" +
78+
" .xlsx (XSSF 2007 OOXML file format) and older .xls (HSSF '97(-2007) file format) Excel documents." +
79+
" Please note all original cell styles are copied and formulas are removed leaving only the calculated values." +
80+
" Even a single sheet Microsoft Excel spreadsheet is converted to its own flow file with all the original cell styles copied and formulas removed."
7381
)
7482
@WritesAttributes({
7583
@WritesAttribute(attribute = "fragment.identifier", description = "All split Excel FlowFiles produced from the same parent Excel FlowFile will have the same randomly generated UUID added" +
@@ -100,6 +108,15 @@ public class SplitExcel extends AbstractProcessor {
100108
.dependsOn(PROTECTION_TYPE, ProtectionType.PASSWORD)
101109
.build();
102110

111+
public static final PropertyDescriptor INPUT_FILE_TYPE = new PropertyDescriptor
112+
.Builder().name("Input File Type")
113+
.displayName("Input File Type")
114+
.description("Specifies type of Excel input file.")
115+
.required(true)
116+
.allowableValues(InputFileType.class)
117+
.defaultValue(InputFileType.XLSX)
118+
.build();
119+
103120
public static final Relationship REL_ORIGINAL = new Relationship.Builder()
104121
.name("original")
105122
.description("The original FlowFile that was split into segments. If the FlowFile fails processing, nothing will be sent to this relationship")
@@ -117,7 +134,8 @@ public class SplitExcel extends AbstractProcessor {
117134

118135
private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = List.of(
119136
PROTECTION_TYPE,
120-
PASSWORD
137+
PASSWORD,
138+
INPUT_FILE_TYPE
121139
);
122140

123141
private static final Set<Relationship> RELATIONSHIPS = Set.of(
@@ -126,17 +144,6 @@ public class SplitExcel extends AbstractProcessor {
126144
REL_SPLIT
127145
);
128146

129-
private static final CellCopyPolicy CELL_COPY_POLICY = new CellCopyPolicy.Builder()
130-
.cellFormula(false) // NOTE: setting to false allows for copying the evaluated formula value.
131-
.cellStyle(CellCopyPolicy.DEFAULT_COPY_CELL_STYLE_POLICY)
132-
.cellValue(CellCopyPolicy.DEFAULT_COPY_CELL_VALUE_POLICY)
133-
.condenseRows(CellCopyPolicy.DEFAULT_CONDENSE_ROWS_POLICY)
134-
.copyHyperlink(false) // NOTE: the hyperlinks appear at end of sheet, so we need to iterate them separately at the end.
135-
.mergeHyperlink(CellCopyPolicy.DEFAULT_MERGE_HYPERLINK_POLICY)
136-
.mergedRegions(false) // NOTE: set to false because of the explicit merge region handling in the copyRows method.
137-
.rowHeight(CellCopyPolicy.DEFAULT_COPY_ROW_HEIGHT_POLICY)
138-
.build();
139-
140147
@Override
141148
public Set<Relationship> getRelationships() {
142149
return RELATIONSHIPS;
@@ -156,35 +163,17 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro
156163

157164
final ProtectionType protectionType = context.getProperty(PROTECTION_TYPE).asAllowableValue(ProtectionType.class);
158165
final String password = protectionType == ProtectionType.PASSWORD ? context.getProperty(PASSWORD).getValue() : null;
166+
final InputFileType inputFileType = context.getProperty(INPUT_FILE_TYPE).asAllowableValue(InputFileType.class);
167+
final CellCopyPolicy cellCopyPolicy = createCellCopyPolicy(inputFileType);
159168
final List<WorkbookSplit> workbookSplits = new ArrayList<>();
160169

161170
try {
162171
session.read(originalFlowFile, in -> {
163172

164-
final Workbook originalWorkbook = StreamingReader.builder()
165-
.rowCacheSize(100)
166-
.bufferSize(4096)
167-
.password(password)
168-
.setReadHyperlinks(true) // NOTE: Needed for copying rows.
169-
.setReadSharedFormulas(true) // NOTE: If not set to true, then data with shared formulas fail.
170-
.open(in);
171-
172-
int index = 0;
173-
for (final Sheet originalSheet : originalWorkbook) {
174-
final String originalSheetName = originalSheet.getSheetName();
175-
176-
try (final SXSSFWorkbook newWorkbook = new SXSSFWorkbook(null, SXSSFWorkbook.DEFAULT_WINDOW_SIZE, false, true)) {
177-
final SXSSFSheet newSheet = newWorkbook.createSheet(originalSheetName);
178-
final int numberOfCopiedRows = copyRows(originalSheet, newSheet);
179-
180-
final FlowFile newFlowFile = session.create(originalFlowFile);
181-
try (final OutputStream out = session.write(newFlowFile)) {
182-
newWorkbook.write(out);
183-
workbookSplits.add(new WorkbookSplit(index, newFlowFile, originalSheetName, numberOfCopiedRows));
184-
}
185-
}
186-
187-
index++;
173+
if (inputFileType == InputFileType.XLSX) {
174+
handleXSSF(session, originalFlowFile, in, password, workbookSplits, cellCopyPolicy);
175+
} else {
176+
handleHSSF(session, originalFlowFile, in, password, workbookSplits, cellCopyPolicy);
188177
}
189178
});
190179
} catch (ExcelRuntimeException | IllegalStateException | ProcessException e) {
@@ -229,7 +218,56 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro
229218
session.transfer(flowFileSplits, REL_SPLIT);
230219
}
231220

232-
private int copyRows(final Sheet originalSheet, final SXSSFSheet destinationSheet) {
221+
private CellCopyPolicy createCellCopyPolicy(InputFileType inputFileType) {
222+
CellCopyPolicy.Builder builder = new CellCopyPolicy.Builder()
223+
.cellFormula(false) // NOTE: setting to false allows for copying the evaluated formula value.
224+
.cellStyle(CellCopyPolicy.DEFAULT_COPY_CELL_STYLE_POLICY)
225+
.cellValue(CellCopyPolicy.DEFAULT_COPY_CELL_VALUE_POLICY)
226+
.condenseRows(CellCopyPolicy.DEFAULT_CONDENSE_ROWS_POLICY)
227+
.mergeHyperlink(CellCopyPolicy.DEFAULT_MERGE_HYPERLINK_POLICY)
228+
.rowHeight(CellCopyPolicy.DEFAULT_COPY_ROW_HEIGHT_POLICY);
229+
230+
if (inputFileType == InputFileType.XLSX) {
231+
builder.copyHyperlink(false) // NOTE: the hyperlinks appear at end of sheet, so we need to iterate them separately at the end.
232+
.mergedRegions(false); // NOTE: set to false because of the explicit merge region handling in the copyRows method.
233+
} else {
234+
builder.copyHyperlink(CellCopyPolicy.DEFAULT_COPY_HYPERLINK_POLICY)
235+
.mergedRegions(CellCopyPolicy.DEFAULT_COPY_MERGED_REGIONS_POLICY);
236+
}
237+
238+
return builder.build();
239+
}
240+
241+
private void handleXSSF(ProcessSession session, FlowFile originalFlowFile, InputStream inputStream, String password,
242+
List<WorkbookSplit> workbookSplits, CellCopyPolicy cellCopyPolicy) throws IOException {
243+
final Workbook originalWorkbook = StreamingReader.builder()
244+
.rowCacheSize(100)
245+
.bufferSize(4096)
246+
.password(password)
247+
.setReadHyperlinks(true) // NOTE: Needed for copying rows.
248+
.setReadSharedFormulas(true) // NOTE: If not set to true, then data with shared formulas fail.
249+
.open(inputStream);
250+
251+
int index = 0;
252+
for (final Sheet originalSheet : originalWorkbook) {
253+
final String originalSheetName = originalSheet.getSheetName();
254+
255+
try (final SXSSFWorkbook newWorkbook = new SXSSFWorkbook(null, SXSSFWorkbook.DEFAULT_WINDOW_SIZE, false, true)) {
256+
final SXSSFSheet newSheet = newWorkbook.createSheet(originalSheetName);
257+
final int numberOfCopiedRows = copyRows(originalSheet, newSheet, cellCopyPolicy);
258+
259+
final FlowFile newFlowFile = session.create(originalFlowFile);
260+
try (final OutputStream out = session.write(newFlowFile)) {
261+
newWorkbook.write(out);
262+
workbookSplits.add(new WorkbookSplit(index, newFlowFile, originalSheetName, numberOfCopiedRows));
263+
}
264+
}
265+
266+
index++;
267+
}
268+
}
269+
270+
private int copyRows(final Sheet originalSheet, final SXSSFSheet destinationSheet, CellCopyPolicy cellCopyPolicy) {
233271
final CellCopyContext cellCopyContext = new CellCopyContext();
234272
int rowCount = 0;
235273

@@ -239,7 +277,7 @@ private int copyRows(final Sheet originalSheet, final SXSSFSheet destinationShee
239277

240278
for (final Cell sourceCell : sourceRow) {
241279
final Cell destCell = destinationRow.createCell(sourceCell.getColumnIndex());
242-
CellUtil.copyCell(sourceCell, destCell, CELL_COPY_POLICY, cellCopyContext);
280+
CellUtil.copyCell(sourceCell, destCell, cellCopyPolicy, cellCopyContext);
243281
}
244282

245283
rowCount++;
@@ -256,5 +294,50 @@ private int copyRows(final Sheet originalSheet, final SXSSFSheet destinationShee
256294
return rowCount;
257295
}
258296

259-
private record WorkbookSplit(int index, FlowFile content, String sheetName, int numRows) { }
297+
private void handleHSSF(ProcessSession session, FlowFile originalFlowFile, InputStream inputStream, String password, List<WorkbookSplit> workbookSplits, CellCopyPolicy cellCopyPolicy) {
298+
// Providing the password to the HSSFWorkbook is done by setting a thread variable managed by
299+
// Biff8EncryptionKey. After the workbook is created, the thread variable can be cleared.
300+
Biff8EncryptionKey.setCurrentUserPassword(password);
301+
302+
try {
303+
final HSSFWorkbook originalWorkbook = new HSSFWorkbook(inputStream);
304+
final Iterator<Sheet> originalSheetsIterator = originalWorkbook.sheetIterator();
305+
final CellCopyContext cellCopyContext = new CellCopyContext();
306+
307+
int index = 0;
308+
while (originalSheetsIterator.hasNext()) {
309+
final HSSFSheet originalSheet = (HSSFSheet) originalSheetsIterator.next();
310+
final String originalSheetName = originalSheet.getSheetName();
311+
//NOTE: Per the POI Javadocs, the rowIterator returns an iterator of the physical rows,
312+
// hence the original number of rows should reflect this.
313+
final int originalNumRows = originalSheet.getPhysicalNumberOfRows();
314+
final Iterator<Row> originalRowsIterator = originalSheet.rowIterator();
315+
316+
try (HSSFWorkbook newWorkbook = new HSSFWorkbook()) {
317+
final HSSFSheet newSheet = newWorkbook.createSheet(originalSheetName);
318+
while (originalRowsIterator.hasNext()) {
319+
HSSFRow originalRow = (HSSFRow) originalRowsIterator.next();
320+
HSSFRow newRow = newSheet.createRow(originalRow.getRowNum());
321+
newRow.copyRowFrom(originalRow, cellCopyPolicy, cellCopyContext);
322+
}
323+
324+
FlowFile newFlowFile = session.create(originalFlowFile);
325+
326+
try (final OutputStream out = session.write(newFlowFile)) {
327+
newWorkbook.write(out);
328+
workbookSplits.add(new WorkbookSplit(index, newFlowFile, originalSheetName, originalNumRows));
329+
}
330+
}
331+
index++;
332+
}
333+
334+
} catch (final IOException e) {
335+
throw new ProcessException("Failed to split XLS file", e);
336+
} finally {
337+
Biff8EncryptionKey.setCurrentUserPassword(null);
338+
}
339+
}
340+
341+
private record WorkbookSplit(int index, FlowFile content, String sheetName, int numRows) {
342+
}
260343
}

nifi-extension-bundles/nifi-poi-bundle/nifi-poi-services/src/test/java/org/apache/nifi/processors/excel/TestSplitExcel.java

Lines changed: 14 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -16,22 +16,26 @@
1616
*/
1717
package org.apache.nifi.processors.excel;
1818

19+
import org.apache.nifi.excel.InputFileType;
1920
import org.apache.nifi.util.MockFlowFile;
2021
import org.apache.nifi.util.TestRunner;
2122
import org.apache.nifi.util.TestRunners;
23+
import org.apache.poi.hssf.usermodel.HSSFWorkbook;
2224
import org.apache.poi.ss.usermodel.Cell;
2325
import org.apache.poi.ss.usermodel.CellStyle;
2426
import org.apache.poi.ss.usermodel.CellType;
2527
import org.apache.poi.ss.usermodel.CreationHelper;
2628
import org.apache.poi.ss.usermodel.DateUtil;
2729
import org.apache.poi.ss.usermodel.Row;
2830
import org.apache.poi.ss.usermodel.Sheet;
31+
import org.apache.poi.ss.usermodel.Workbook;
2932
import org.apache.poi.xssf.usermodel.XSSFHyperlink;
30-
import org.apache.poi.xssf.usermodel.XSSFSheet;
3133
import org.apache.poi.xssf.usermodel.XSSFWorkbook;
3234
import org.junit.jupiter.api.AfterAll;
3335
import org.junit.jupiter.api.BeforeEach;
3436
import org.junit.jupiter.api.Test;
37+
import org.junit.jupiter.params.ParameterizedTest;
38+
import org.junit.jupiter.params.provider.EnumSource;
3539

3640
import java.io.ByteArrayInputStream;
3741
import java.io.ByteArrayOutputStream;
@@ -198,8 +202,9 @@ void testDataWithSharedFormula() throws IOException {
198202
}
199203
}
200204

201-
@Test
202-
void testCopyDateTime() throws Exception {
205+
@ParameterizedTest
206+
@EnumSource(InputFileType.class)
207+
void testCopyDateTime(InputFileType inputFileType) throws Exception {
203208
final LocalDateTime localDateTime = LocalDateTime.of(2023, 1, 1, 0, 0, 0);
204209
final LocalDateTime nonValidExcelDate = LocalDateTime.of(1899, 12, 31, 0, 0, 0);
205210

@@ -210,14 +215,15 @@ void testCopyDateTime() throws Exception {
210215
};
211216

212217
final ByteArrayOutputStream workbookOutputStream = new ByteArrayOutputStream();
213-
try (XSSFWorkbook workbook = new XSSFWorkbook()) {
214-
final XSSFSheet sheet = workbook.createSheet("SomeSheetName");
218+
try (Workbook workbook = InputFileType.XLSX == inputFileType ? new XSSFWorkbook() : new HSSFWorkbook()) {
219+
final Sheet sheet = workbook.createSheet("SomeSheetName");
215220
populateSheet(sheet, data);
216221
setCellStyles(sheet, workbook);
217222
workbook.write(workbookOutputStream);
218223
}
219224

220225
final ByteArrayInputStream input = new ByteArrayInputStream(workbookOutputStream.toByteArray());
226+
runner.setProperty(SplitExcel.INPUT_FILE_TYPE, inputFileType.getValue());
221227
runner.enqueue(input);
222228
runner.run();
223229

@@ -226,7 +232,7 @@ void testCopyDateTime() throws Exception {
226232
runner.assertTransferCount(SplitExcel.REL_FAILURE, 0);
227233

228234
final MockFlowFile flowFile = runner.getFlowFilesForRelationship(SplitExcel.REL_SPLIT).getFirst();
229-
try (XSSFWorkbook workbook = new XSSFWorkbook(flowFile.getContentStream())) {
235+
try (Workbook workbook = InputFileType.XLSX == inputFileType ? new XSSFWorkbook(flowFile.getContentStream()) : new HSSFWorkbook(flowFile.getContentStream())) {
230236
final Sheet firstSheet = workbook.sheetIterator().next();
231237

232238
List<List<Cell>> dateCells = Stream.iterate(firstSheet.getFirstRowNum() + 1, rowIndex -> rowIndex + 1)
@@ -273,7 +279,7 @@ void testHyperlinks() throws IOException {
273279
}
274280
}
275281

276-
private static void populateSheet(XSSFSheet sheet, Object[][] data) {
282+
private static void populateSheet(Sheet sheet, Object[][] data) {
277283
int rowCount = 0;
278284
for (Object[] dataRow : data) {
279285
Row row = sheet.createRow(rowCount++);
@@ -293,7 +299,7 @@ private static void populateSheet(XSSFSheet sheet, Object[][] data) {
293299
}
294300
}
295301

296-
void setCellStyles(XSSFSheet sheet, XSSFWorkbook workbook) {
302+
void setCellStyles(Sheet sheet, Workbook workbook) {
297303
CreationHelper creationHelper = workbook.getCreationHelper();
298304
CellStyle dayMonthYearCellStyle = workbook.createCellStyle();
299305
dayMonthYearCellStyle.setDataFormat(creationHelper.createDataFormat().getFormat("dd/mm/yyyy"));

0 commit comments

Comments
 (0)