-
Notifications
You must be signed in to change notification settings - Fork 703
/
LocalCarbonFile.java
478 lines (419 loc) · 16.5 KB
/
LocalCarbonFile.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
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
/*
* 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.carbondata.core.datastore.filesystem;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileFilter;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.datastore.impl.FileFactory;
import org.apache.carbondata.core.util.CarbonUtil;
import com.github.luben.zstd.ZstdInputStream;
import com.github.luben.zstd.ZstdOutputStream;
import net.jpountz.lz4.LZ4BlockInputStream;
import net.jpountz.lz4.LZ4BlockOutputStream;
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.permission.FsPermission;
import org.xerial.snappy.SnappyInputStream;
import org.xerial.snappy.SnappyOutputStream;
public class LocalCarbonFile implements CarbonFile {
private static final LogService LOGGER =
LogServiceFactory.getLogService(LocalCarbonFile.class.getName());
private File file;
public LocalCarbonFile(String filePath) {
Path pathWithoutSchemeAndAuthority = Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
file = new File(pathWithoutSchemeAndAuthority.toString());
}
public LocalCarbonFile(File file) {
this.file = file;
}
@Override public String getAbsolutePath() {
return file.getAbsolutePath();
}
@Override public CarbonFile[] listFiles(final CarbonFileFilter fileFilter) {
if (!file.isDirectory()) {
return new CarbonFile[0];
}
File[] files = file.listFiles(new FileFilter() {
@Override public boolean accept(File pathname) {
return fileFilter.accept(new LocalCarbonFile(pathname));
}
});
if (files == null) {
return new CarbonFile[0];
}
CarbonFile[] carbonFiles = new CarbonFile[files.length];
for (int i = 0; i < carbonFiles.length; i++) {
carbonFiles[i] = new LocalCarbonFile(files[i]);
}
return carbonFiles;
}
@Override public String getName() {
return file.getName();
}
@Override public boolean isDirectory() {
return file.isDirectory();
}
@Override public boolean exists() {
if (file != null) {
return file.exists();
}
return false;
}
@Override public String getCanonicalPath() {
try {
return file.getCanonicalPath();
} catch (IOException e) {
LOGGER
.error(e, "Exception occured" + e.getMessage());
}
return null;
}
@Override public CarbonFile getParentFile() {
return new LocalCarbonFile(file.getParentFile());
}
@Override public String getPath() {
return file.getPath();
}
@Override public long getSize() {
return file.length();
}
public boolean renameTo(String changetoName) {
changetoName = FileFactory.getUpdatedFilePath(changetoName, FileFactory.FileType.LOCAL);
return file.renameTo(new File(changetoName));
}
public boolean delete() {
return file.delete();
}
@Override public CarbonFile[] listFiles() {
if (!file.isDirectory()) {
return new CarbonFile[0];
}
File[] files = file.listFiles();
if (files == null) {
return new CarbonFile[0];
}
CarbonFile[] carbonFiles = new CarbonFile[files.length];
for (int i = 0; i < carbonFiles.length; i++) {
carbonFiles[i] = new LocalCarbonFile(files[i]);
}
return carbonFiles;
}
@Override
public List<CarbonFile> listFiles(Boolean recurssive) {
if (!file.isDirectory()) {
return new ArrayList<CarbonFile>();
}
Collection<File> fileCollection = FileUtils.listFiles(file, null, true);
if (fileCollection == null) {
return new ArrayList<CarbonFile>();
}
List<CarbonFile> carbonFiles = new ArrayList<CarbonFile>();
for (File file : fileCollection) {
carbonFiles.add(new LocalCarbonFile(file));
}
return carbonFiles;
}
@Override public boolean createNewFile() {
try {
return file.createNewFile();
} catch (IOException e) {
return false;
}
}
@Override public long getLastModifiedTime() {
return file.lastModified();
}
@Override public boolean setLastModifiedTime(long timestamp) {
return file.setLastModified(timestamp);
}
/**
* This method will delete the data in file data from a given offset
*/
@Override public boolean truncate(String fileName, long validDataEndOffset) {
FileChannel source = null;
FileChannel destination = null;
boolean fileTruncatedSuccessfully = false;
// temporary file name
String tempWriteFilePath = fileName + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
FileFactory.FileType fileType = FileFactory.getFileType(fileName);
try {
CarbonFile tempFile = null;
// delete temporary file if it already exists at a given path
if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
tempFile.delete();
}
// create new temporary file
FileFactory.createNewFile(tempWriteFilePath, fileType);
tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
source = new FileInputStream(fileName).getChannel();
destination = new FileOutputStream(tempWriteFilePath).getChannel();
long read = destination.transferFrom(source, 0, validDataEndOffset);
long totalBytesRead = read;
long remaining = validDataEndOffset - totalBytesRead;
// read till required data offset is not reached
while (remaining > 0) {
read = destination.transferFrom(source, totalBytesRead, remaining);
totalBytesRead = totalBytesRead + read;
remaining = remaining - totalBytesRead;
}
CarbonUtil.closeStreams(source, destination);
// rename the temp file to original file
tempFile.renameForce(fileName);
fileTruncatedSuccessfully = true;
} catch (IOException e) {
LOGGER.error("Exception occured while truncating the file " + e.getMessage());
} finally {
CarbonUtil.closeStreams(source, destination);
}
return fileTruncatedSuccessfully;
}
/**
* This method will be used to check whether a file has been modified or not
*
* @param fileTimeStamp time to be compared with latest timestamp of file
* @param endOffset file length to be compared with current length of file
* @return
*/
@Override public boolean isFileModified(long fileTimeStamp, long endOffset) {
boolean isFileModified = false;
if (getLastModifiedTime() > fileTimeStamp || getSize() > endOffset) {
isFileModified = true;
}
return isFileModified;
}
@Override public boolean renameForce(String changetoName) {
File destFile = new File(changetoName);
if (destFile.exists() && !file.getAbsolutePath().equals(destFile.getAbsolutePath())) {
if (destFile.delete()) {
return file.renameTo(new File(changetoName));
}
}
return file.renameTo(new File(changetoName));
}
@Override public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType,
int bufferSize, boolean append) throws FileNotFoundException {
path = FileFactory.getUpdatedFilePath(path, FileFactory.FileType.LOCAL);
return new DataOutputStream(
new BufferedOutputStream(new FileOutputStream(path, append), bufferSize));
}
@Override public DataInputStream getDataInputStream(String path, FileFactory.FileType fileType,
int bufferSize, Configuration configuration) throws IOException {
return getDataInputStream(path, fileType, bufferSize,
CarbonUtil.inferCompressorFromFileName(path));
}
@Override public DataInputStream getDataInputStream(String path, FileFactory.FileType fileType,
int bufferSize, String compressor) throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, fileType);
InputStream inputStream;
if (compressor.isEmpty()) {
inputStream = new FileInputStream(path);
} else if ("GZIP".equalsIgnoreCase(compressor)) {
inputStream = new GZIPInputStream(new FileInputStream(path));
} else if ("BZIP2".equalsIgnoreCase(compressor)) {
inputStream = new BZip2CompressorInputStream(new FileInputStream(path));
} else if ("SNAPPY".equalsIgnoreCase(compressor)) {
inputStream = new SnappyInputStream(new FileInputStream(path));
} else if ("LZ4".equalsIgnoreCase(compressor)) {
inputStream = new LZ4BlockInputStream(new FileInputStream(path));
} else if ("ZSTD".equalsIgnoreCase(compressor)) {
inputStream = new ZstdInputStream(new FileInputStream(path));
} else {
throw new IOException("Unsupported compressor: " + compressor);
}
if (bufferSize <= 0) {
return new DataInputStream(new BufferedInputStream(inputStream));
} else {
return new DataInputStream(new BufferedInputStream(inputStream, bufferSize));
}
}
/**
* return the datainputStream which is seek to the offset of file
*
* @param path
* @param fileType
* @param bufferSize
* @param offset
* @return DataInputStream
* @throws IOException
*/
@Override public DataInputStream getDataInputStream(String path, FileFactory.FileType fileType,
int bufferSize, long offset) throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, fileType);
FileInputStream fis = new FileInputStream(path);
long actualSkipSize = 0;
long skipSize = offset;
try {
while (actualSkipSize != offset) {
actualSkipSize += fis.skip(skipSize);
skipSize = skipSize - actualSkipSize;
}
} catch (IOException ioe) {
CarbonUtil.closeStream(fis);
throw ioe;
}
return new DataInputStream(new BufferedInputStream(fis));
}
@Override
public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType)
throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, FileFactory.FileType.LOCAL);
return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path)));
}
@Override
public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType,
int bufferSize, long blockSize) throws IOException {
return getDataOutputStream(path, fileType, bufferSize, blockSize, (short) 1);
}
@Override
public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType,
int bufferSize, long blockSize, short replication) throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, fileType);
return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path), bufferSize));
}
@Override
public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType,
int bufferSize, String compressor) throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, fileType);
OutputStream outputStream;
if (compressor.isEmpty()) {
outputStream = new FileOutputStream(path);
} else if ("GZIP".equalsIgnoreCase(compressor)) {
outputStream = new GZIPOutputStream(new FileOutputStream(path));
} else if ("BZIP2".equalsIgnoreCase(compressor)) {
outputStream = new BZip2CompressorOutputStream(new FileOutputStream(path));
} else if ("SNAPPY".equalsIgnoreCase(compressor)) {
outputStream = new SnappyOutputStream(new FileOutputStream(path));
} else if ("LZ4".equalsIgnoreCase(compressor)) {
outputStream = new LZ4BlockOutputStream(new FileOutputStream(path));
} else if ("ZSTD".equalsIgnoreCase(compressor)) {
// compression level 1 is cost-effective for sort temp file
// which is not used for storage
outputStream = new ZstdOutputStream(new FileOutputStream(path), 1);
} else {
throw new IOException("Unsupported compressor: " + compressor);
}
if (bufferSize <= 0) {
return new DataOutputStream(new BufferedOutputStream(outputStream));
} else {
return new DataOutputStream(new BufferedOutputStream(outputStream, bufferSize));
}
}
@Override public boolean isFileExist(String filePath, FileFactory.FileType fileType,
boolean performFileCheck) throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File defaultFile = new File(filePath);
if (performFileCheck) {
return defaultFile.exists() && defaultFile.isFile();
} else {
return defaultFile.exists();
}
}
@Override public boolean isFileExist(String filePath, FileFactory.FileType fileType)
throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File defaultFile = new File(filePath);
return defaultFile.exists();
}
@Override public boolean createNewFile(String filePath, FileFactory.FileType fileType)
throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File file = new File(filePath);
return file.createNewFile();
}
@Override
public boolean createNewFile(String filePath, FileFactory.FileType fileType, boolean doAs,
final FsPermission permission) throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File file = new File(filePath);
return file.createNewFile();
}
@Override public boolean deleteFile(String filePath, FileFactory.FileType fileType)
throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File file = new File(filePath);
return FileFactory.deleteAllFilesOfDir(file);
}
@Override public boolean mkdirs(String filePath)
throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath);
File file = new File(filePath);
return file.mkdirs();
}
@Override
public DataOutputStream getDataOutputStreamUsingAppend(String path, FileFactory.FileType fileType)
throws IOException {
path = path.replace("\\", "/");
path = FileFactory.getUpdatedFilePath(path, fileType);
return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path, true)));
}
@Override public boolean createNewLockFile(String filePath, FileFactory.FileType fileType)
throws IOException {
filePath = filePath.replace("\\", "/");
filePath = FileFactory.getUpdatedFilePath(filePath, fileType);
File file = new File(filePath);
return file.createNewFile();
}
@Override public CarbonFile[] locationAwareListFiles(PathFilter pathFilter) throws IOException {
return listFiles();
}
@Override public String[] getLocations() throws IOException {
return new String[]{"localhost"};
}
@Override
public boolean setReplication(String filePath, short replication) throws IOException {
// local carbon file does not need replication
return true;
}
@Override
public short getDefaultReplication(String filePath) throws IOException {
return 1;
}
}