-
Notifications
You must be signed in to change notification settings - Fork 457
/
WriteSPInstruction.java
383 lines (333 loc) · 14.4 KB
/
WriteSPInstruction.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
/*
* 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.sysds.runtime.instructions.spark;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Random;
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.util.LongAccumulator;
import org.apache.sysds.common.Types.DataType;
import org.apache.sysds.common.Types.FileFormat;
import org.apache.sysds.common.Types.ValueType;
import org.apache.sysds.conf.ConfigurationManager;
import org.apache.sysds.runtime.DMLRuntimeException;
import org.apache.sysds.runtime.compress.io.WriterCompressed;
import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
import org.apache.sysds.runtime.controlprogram.context.SparkExecutionContext;
import org.apache.sysds.runtime.frame.data.FrameBlock;
import org.apache.sysds.runtime.instructions.InstructionUtils;
import org.apache.sysds.runtime.instructions.cp.CPOperand;
import org.apache.sysds.runtime.instructions.spark.functions.ComputeBinaryBlockNnzFunction;
import org.apache.sysds.runtime.instructions.spark.utils.FrameRDDConverterUtils;
import org.apache.sysds.runtime.instructions.spark.utils.FrameRDDConverterUtils.LongFrameToLongWritableFrameFunction;
import org.apache.sysds.runtime.instructions.spark.utils.RDDConverterUtils;
import org.apache.sysds.runtime.io.FileFormatProperties;
import org.apache.sysds.runtime.io.FileFormatPropertiesCSV;
import org.apache.sysds.runtime.io.FileFormatPropertiesLIBSVM;
import org.apache.sysds.runtime.lineage.LineageItem;
import org.apache.sysds.runtime.lineage.LineageItemUtils;
import org.apache.sysds.runtime.lineage.LineageTraceable;
import org.apache.sysds.runtime.matrix.data.MatrixBlock;
import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
import org.apache.sysds.runtime.meta.DataCharacteristics;
import org.apache.sysds.runtime.meta.MatrixCharacteristics;
import org.apache.sysds.runtime.util.HDFSTool;
public class WriteSPInstruction extends SPInstruction implements LineageTraceable {
public CPOperand input1 = null;
private CPOperand input2 = null;
private CPOperand input3 = null;
private CPOperand input4 = null;
private FileFormatProperties formatProperties;
private WriteSPInstruction(CPOperand in1, CPOperand in2, CPOperand in3, String opcode, String str) {
super(SPType.Write, opcode, str);
input1 = in1;
input2 = in2;
input3 = in3;
formatProperties = null; // set in case of csv
}
public static WriteSPInstruction parseInstruction ( String str ) {
String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
String opcode = parts[0];
if( !opcode.equals("write") ) {
throw new DMLRuntimeException("Unsupported opcode");
}
// All write instructions have 3 parameters, except in case of delimited/csv/libsvm file.
// Write instructions for csv files also include three additional parameters (hasHeader, delimiter, sparse)
// Write instructions for libsvm files also include three additional parameters (delimiter, index delimiter, sparse)
if ( parts.length != 6 && parts.length != 10 ) {
throw new DMLRuntimeException("Invalid number of operands in write instruction: " + str);
}
// _mVar2·MATRIX·DOUBLE
CPOperand in1 = new CPOperand(parts[1]);
CPOperand in2 = new CPOperand(parts[2]);
CPOperand in3 = new CPOperand(parts[3]);
WriteSPInstruction inst = new WriteSPInstruction(in1, in2, in3, opcode, str);
if ( in3.getName().equalsIgnoreCase("csv") ) {
boolean hasHeader = Boolean.parseBoolean(parts[4]);
String delim = parts[5];
boolean sparse = Boolean.parseBoolean(parts[6]);
FileFormatProperties formatProperties = new FileFormatPropertiesCSV(hasHeader, delim, sparse);
inst.setFormatProperties(formatProperties);
CPOperand in4 = new CPOperand(parts[8]);
inst.input4 = in4;
}
else if(in3.getName().equalsIgnoreCase("libsvm")) {
String delim = parts[4];
String indexDelim = parts[5];
boolean sparse = Boolean.parseBoolean(parts[6]);
FileFormatProperties formatProperties = new FileFormatPropertiesLIBSVM(delim, indexDelim, sparse);
inst.setFormatProperties(formatProperties);
CPOperand in4 = new CPOperand(parts[8]);
inst.input4 = in4;
}
else {
FileFormatProperties ffp = new FileFormatProperties();
CPOperand in4 = new CPOperand(parts[5]);
inst.input4 = in4;
inst.setFormatProperties(ffp);
}
return inst;
}
public FileFormatProperties getFormatProperties() {
return formatProperties;
}
public void setFormatProperties(FileFormatProperties prop) {
formatProperties = prop;
}
public CPOperand getInput1() {
return input1;
}
public CPOperand getInput2() {
return input2;
}
@Override
public void processInstruction(ExecutionContext ec) {
SparkExecutionContext sec = (SparkExecutionContext) ec;
//get filename (literal or variable expression)
String fname = ec.getScalarInput(input2).getStringValue();
String desc = ec.getScalarInput(input4).getStringValue();
formatProperties.setDescription(desc);
ValueType[] schema = (input1.getDataType()==DataType.FRAME) ?
sec.getFrameObject(input1.getName()).getSchema() : null;
try
{
//if the file already exists on HDFS, remove it.
HDFSTool.deleteFileIfExistOnHDFS( fname );
//prepare output info according to meta data
String fmtStr = ec.getScalarInput(input3).getStringValue();
FileFormat fmt = FileFormat.safeValueOf(fmtStr);
//core matrix/frame write
switch( input1.getDataType() ) {
case MATRIX: processMatrixWriteInstruction(sec, fname, fmt); break;
case FRAME: processFrameWriteInstruction(sec, fname, fmt, schema); break;
default: throw new DMLRuntimeException(
"Unsupported data type "+input1.getDataType()+" in WriteSPInstruction.");
}
}
catch(IOException ex)
{
throw new DMLRuntimeException("Failed to process write instruction", ex);
}
}
protected void processMatrixWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt)
throws IOException
{
//get input rdd
JavaPairRDD<MatrixIndexes,MatrixBlock> in1 = sec.getBinaryMatrixBlockRDDHandleForVariable( input1.getName() );
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
DataCharacteristics mcOut = mc; //by reference
if( fmt == FileFormat.MM || fmt == FileFormat.TEXT ) {
//piggyback nnz maintenance on write
LongAccumulator aNnz = null;
if( !mc.nnzKnown() ) {
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
JavaRDD<String> header = null;
if( fmt == FileFormat.MM ) {
ArrayList<String> headerContainer = new ArrayList<>(1);
// First output MM header
String headerStr = "%%MatrixMarket matrix coordinate real general\n" +
// output number of rows, number of columns and number of nnz
mc.getRows() + " " + mc.getCols() + " " + mc.getNonZeros();
headerContainer.add(headerStr);
header = sec.getSparkContext().parallelize(headerContainer);
}
JavaRDD<String> ijv = RDDConverterUtils.binaryBlockToTextCell(in1, mc);
if(header != null)
customSaveTextFile(header.union(ijv), fname, true);
else
customSaveTextFile(ijv, fname, false);
if( !mc.nnzKnown() )
mc.setNonZeros( aNnz.value() );
}
else if( fmt == FileFormat.CSV ) {
if( mc.getRows() == 0 || mc.getCols() == 0 ) {
throw new IOException("Write of matrices with zero rows or columns"
+ " not supported ("+mc.getRows()+"x"+mc.getCols()+").");
}
FileFormatProperties fprop = (formatProperties instanceof FileFormatPropertiesCSV) ?
formatProperties : new FileFormatPropertiesCSV(); //for dynamic format strings
//piggyback nnz computation on actual write
LongAccumulator aNnz = null;
if( !mc.nnzKnown() ) {
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
JavaRDD<String> out = RDDConverterUtils.binaryBlockToCsv(
in1, mc, (FileFormatPropertiesCSV) fprop, true);
customSaveTextFile(out, fname, false);
if( !mc.nnzKnown() )
mc.setNonZeros(aNnz.value().longValue());
}
else if( fmt == FileFormat.BINARY ) {
//reblock output if needed
int blen = Integer.parseInt(input4.getName());
boolean nonDefaultBlen = ConfigurationManager.getBlocksize() != blen && blen > 0;
if( nonDefaultBlen )
in1 = RDDConverterUtils.binaryBlockToBinaryBlock(in1, mc,
new MatrixCharacteristics(mc).setBlocksize(blen));
//piggyback nnz computation on actual write
LongAccumulator aNnz = null;
if( !mc.nnzKnown() ) {
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
//save binary block rdd on hdfs
in1.saveAsHadoopFile(fname, MatrixIndexes.class, MatrixBlock.class, SequenceFileOutputFormat.class);
if( !mc.nnzKnown() ) //update nnz
mc.setNonZeros(aNnz.value().longValue());
if( nonDefaultBlen )
mcOut = new MatrixCharacteristics(mc).setBlocksize(blen);
}
else if(fmt == FileFormat.COMPRESSED) {
// reblock output if needed
final int blen = Integer.parseInt(input4.getName());
final boolean nonDefaultBlen = ConfigurationManager.getBlocksize() != blen;
mc.setNonZeros(-1); // default to unknown non zeros for compressed matrix block
if(nonDefaultBlen)
WriterCompressed.writeRDDToHDFS(in1, fname, blen, mc);
else
WriterCompressed.writeRDDToHDFS(in1, fname);
if(nonDefaultBlen)
mcOut = new MatrixCharacteristics(mc).setBlocksize(blen);
}
else if(fmt == FileFormat.LIBSVM) {
if(mc.getRows() == 0 || mc.getCols() == 0) {
throw new IOException(
"Write of matrices with zero rows or columns" + " not supported (" + mc.getRows() + "x" + mc
.getCols() + ").");
}
//piggyback nnz computation on actual write
LongAccumulator aNnz = null;
if(!mc.nnzKnown()) {
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
JavaRDD<String> out = RDDConverterUtils.binaryBlockToLibsvm(in1,
mc, (FileFormatPropertiesLIBSVM) formatProperties, true);
customSaveTextFile(out, fname, false);
if( !mc.nnzKnown() )
mc.setNonZeros(aNnz.value().longValue());
}
else {
//unsupported formats: binarycell (not externalized)
throw new DMLRuntimeException("Unexpected data format: " + fmt.toString());
}
// write meta data file
HDFSTool.writeMetaDataFile(fname + ".mtd", ValueType.FP64, mcOut, fmt, formatProperties);
}
protected void processFrameWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt, ValueType[] schema)
throws IOException
{
//get input rdd
JavaPairRDD<Long,FrameBlock> in1 = sec
.getFrameBinaryBlockRDDHandleForVariable(input1.getName());
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
switch(fmt) {
case TEXT: {
JavaRDD<String> out = FrameRDDConverterUtils.binaryBlockToTextCell(in1, mc);
customSaveTextFile(out, fname, false);
break;
}
case CSV: {
FileFormatPropertiesCSV props = (formatProperties!=null) ?(FileFormatPropertiesCSV) formatProperties : null;
JavaRDD<String> out = FrameRDDConverterUtils.binaryBlockToCsv(in1, mc, props, true);
customSaveTextFile(out, fname, false);
break;
}
case LIBSVM: {
// TODO: implement for libsvm
// FileFormatPropertiesCSV props = (formatProperties!=null) ?(FileFormatPropertiesCSV) formatProperties : null;
// JavaRDD<String> out = FrameRDDConverterUtils.binaryBlockToCsv(in1, mc, props, true);
// customSaveTextFile(out, fname, false);
break;
}
case BINARY: {
JavaPairRDD<LongWritable,FrameBlock> out = in1.mapToPair(new LongFrameToLongWritableFrameFunction());
out.saveAsHadoopFile(fname, LongWritable.class, FrameBlock.class, SequenceFileOutputFormat.class);
break;
}
default:
throw new DMLRuntimeException("Unexpected data format: " + fmt.toString());
}
// write meta data file
HDFSTool.writeMetaDataFile(fname + ".mtd", input1.getValueType(), schema, DataType.FRAME, mc, fmt, formatProperties);
}
private static void customSaveTextFile(JavaRDD<String> rdd, String fname, boolean inSingleFile) {
if(inSingleFile) {
Random rand = new Random();
String randFName = fname + "_" + rand.nextLong() + "_" + rand.nextLong();
try {
while(HDFSTool.existsFileOnHDFS(randFName)) {
randFName = fname + "_" + rand.nextLong() + "_" + rand.nextLong();
}
rdd.saveAsTextFile(randFName);
HDFSTool.mergeIntoSingleFile(randFName, fname); // Faster version :)
// rdd.coalesce(1, true).saveAsTextFile(randFName);
// MapReduceTool.copyFileOnHDFS(randFName + "/part-00000", fname);
} catch (IOException e) {
throw new DMLRuntimeException("Cannot merge the output into single file: " + e.getMessage());
}
finally {
try {
// This is to make sure that we donot create random files on HDFS
HDFSTool.deleteFileIfExistOnHDFS( randFName );
} catch (IOException e) {
throw new DMLRuntimeException("Cannot merge the output into single file: " + e.getMessage());
}
}
}
else {
rdd.saveAsTextFile(fname);
}
}
@Override
public Pair<String, LineageItem> getLineageItem(ExecutionContext ec) {
LineageItem[] ret = LineageItemUtils.getLineage(ec, input1, input2, input3, input4);
if (formatProperties != null && formatProperties.getDescription() != null && !formatProperties.getDescription().isEmpty())
ret = (LineageItem[])ArrayUtils.add(ret, new LineageItem(formatProperties.getDescription()));
return Pair.of(input1.getName(), new LineageItem(getOpcode(), ret));
}
}