-
Notifications
You must be signed in to change notification settings - Fork 703
/
FileFactory.java
501 lines (462 loc) · 16.1 KB
/
FileFactory.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
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
/*
* 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.impl;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.lang.reflect.Method;
import java.nio.channels.FileChannel;
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.FileReader;
import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
public final class FileFactory {
/**
* LOGGER
*/
private static final LogService LOGGER =
LogServiceFactory.getLogService(FileFactory.class.getName());
private static Configuration configuration = null;
static {
configuration = new Configuration();
configuration.addResource(new Path("../core-default.xml"));
}
private static FileTypeInterface fileFileTypeInterface = new DefaultFileTypeProvider();
public static void setFileTypeInterface(FileTypeInterface fileTypeInterface) {
fileFileTypeInterface = fileTypeInterface;
}
private FileFactory() {
}
public static Configuration getConfiguration() {
return configuration;
}
public static FileReader getFileHolder(FileType fileType) {
return fileFileTypeInterface.getFileHolder(fileType);
}
public static FileType getFileType(String path) {
String lowerPath = path.toLowerCase();
if (lowerPath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX)) {
return FileType.HDFS;
} else if (lowerPath.startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX)) {
return FileType.ALLUXIO;
} else if (lowerPath.startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX)) {
return FileType.VIEWFS;
} else if (lowerPath.startsWith(CarbonCommonConstants.S3N_PREFIX) ||
lowerPath.startsWith(CarbonCommonConstants.S3A_PREFIX) ||
lowerPath.startsWith(CarbonCommonConstants.S3_PREFIX)) {
return FileType.S3;
}
return FileType.LOCAL;
}
public static CarbonFile getCarbonFile(String path) {
return fileFileTypeInterface.getCarbonFile(path, getFileType(path));
}
public static CarbonFile getCarbonFile(String path, FileType fileType) {
return fileFileTypeInterface.getCarbonFile(path, fileType);
}
public static CarbonFile getCarbonFile(String path, FileType fileType,
Configuration hadoopConf) {
return fileFileTypeInterface.getCarbonFile(path, fileType, hadoopConf);
}
public static DataInputStream getDataInputStream(String path, FileType fileType)
throws IOException {
return getDataInputStream(path, fileType, -1);
}
public static DataInputStream getDataInputStream(String path, FileType fileType, int bufferSize)
throws IOException {
return getDataInputStream(path, fileType, bufferSize, configuration);
}
public static DataInputStream getDataInputStream(String path, FileType fileType, int bufferSize,
Configuration configuration) throws IOException {
return getCarbonFile(path).getDataInputStream(path, fileType, bufferSize, configuration);
}
/**
* get data input stream
* @param path
* @param fileType
* @param bufferSize
* @param compressorName name of compressor to read this file
* @return data input stream
* @throws IOException
*/
public static DataInputStream getDataInputStream(String path, FileType fileType, int bufferSize,
String compressorName) throws IOException {
return getCarbonFile(path).getDataInputStream(path, fileType, bufferSize, compressorName);
}
/**
* return the datainputStream which is seek to the offset of file
*
* @param path
* @param fileType
* @param bufferSize
* @param offset
* @return DataInputStream
* @throws IOException
*/
public static DataInputStream getDataInputStream(String path, FileType fileType, int bufferSize,
long offset) throws IOException {
return getCarbonFile(path).getDataInputStream(path, fileType, bufferSize, offset);
}
public static DataOutputStream getDataOutputStream(String path, FileType fileType)
throws IOException {
return getCarbonFile(path).getDataOutputStream(path, fileType);
}
public static DataOutputStream getDataOutputStream(String path, FileType fileType, int bufferSize,
boolean append) throws IOException {
return getCarbonFile(path).getDataOutputStream(path, fileType, bufferSize, append);
}
public static DataOutputStream getDataOutputStream(String path, FileType fileType, int bufferSize,
long blockSize) throws IOException {
return getCarbonFile(path).getDataOutputStream(path, fileType, bufferSize, blockSize);
}
/**
* get data output stream
* @param path file path
* @param fileType file type
* @param bufferSize write buffer size
* @param blockSize block size
* @param replication replication
* @return data output stream
* @throws IOException if error occurs
*/
public static DataOutputStream getDataOutputStream(String path, FileType fileType, int bufferSize,
long blockSize, short replication) throws IOException {
return getCarbonFile(path).getDataOutputStream(path, fileType, bufferSize, blockSize,
replication);
}
/**
* get data out put stream
* @param path
* @param fileType
* @param bufferSize
* @param compressorName name of compressor to write this file
* @return data out put stram
* @throws IOException
*/
public static DataOutputStream getDataOutputStream(String path, FileType fileType, int bufferSize,
String compressorName) throws IOException {
return getCarbonFile(path).getDataOutputStream(path, fileType, bufferSize, compressorName);
}
/**
* This method checks the given path exists or not and also is it file or
* not if the performFileCheck is true
*
* @param filePath - Path
* @param fileType - FileType Local/HDFS
* @param performFileCheck - Provide false for folders, true for files and
*/
public static boolean isFileExist(String filePath, FileType fileType, boolean performFileCheck)
throws IOException {
return getCarbonFile(filePath).isFileExist(filePath, fileType, performFileCheck);
}
/**
* This method checks the given path exists or not.
*
* @param filePath - Path
* @param fileType - FileType Local/HDFS
*/
public static boolean isFileExist(String filePath, FileType fileType) throws IOException {
return getCarbonFile(filePath).isFileExist(filePath, fileType);
}
/**
* This method checks the given path exists or not.
*
* @param filePath - Path
*/
public static boolean isFileExist(String filePath) throws IOException {
return isFileExist(filePath, getFileType(filePath));
}
public static boolean createNewFile(String filePath, FileType fileType) throws IOException {
return createNewFile(filePath, fileType, true, null);
}
public static boolean createNewFile(
String filePath,
FileType fileType,
boolean doAs,
final FsPermission permission) throws IOException {
return getCarbonFile(filePath).createNewFile(filePath, fileType, doAs, permission);
}
public static boolean deleteFile(String filePath, FileType fileType) throws IOException {
return getCarbonFile(filePath).deleteFile(filePath, fileType);
}
public static boolean deleteAllFilesOfDir(File path) {
if (!path.exists()) {
return true;
}
if (path.isFile()) {
return path.delete();
}
File[] files = path.listFiles();
if (null == files) {
return true;
}
for (int i = 0; i < files.length; i++) {
deleteAllFilesOfDir(files[i]);
}
return path.delete();
}
public static boolean deleteAllCarbonFilesOfDir(CarbonFile path) {
if (!path.exists()) {
return true;
}
if (!path.isDirectory()) {
return path.delete();
}
CarbonFile[] files = path.listFiles();
for (int i = 0; i < files.length; i++) {
deleteAllCarbonFilesOfDir(files[i]);
}
return path.delete();
}
public static boolean mkdirs(String filePath, FileType fileType) throws IOException {
return getCarbonFile(filePath).mkdirs(filePath);
}
/**
* for getting the dataoutput stream using the hdfs filesystem append API.
*
* @param path
* @param fileType
* @return
* @throws IOException
*/
public static DataOutputStream getDataOutputStreamUsingAppend(String path, FileType fileType)
throws IOException {
return getCarbonFile(path).getDataOutputStreamUsingAppend(path, fileType);
}
/**
* this method will truncate the file to the new size.
* @param path
* @param fileType
* @param newSize
* @throws IOException
*/
public static void truncateFile(String path, FileType fileType, long newSize) throws IOException {
path = path.replace("\\", "/");
FileChannel fileChannel = null;
switch (fileType) {
case LOCAL:
path = getUpdatedFilePath(path, fileType);
fileChannel = new FileOutputStream(path, true).getChannel();
try {
fileChannel.truncate(newSize);
} finally {
if (fileChannel != null) {
fileChannel.close();
}
}
return;
case HDFS:
case ALLUXIO:
case VIEWFS:
case S3:
// if hadoop version >= 2.7, it can call method 'FileSystem.truncate' to truncate file,
// this method was new in hadoop 2.7, otherwise use CarbonFile.truncate to do this.
try {
Path pt = new Path(path);
FileSystem fs = pt.getFileSystem(configuration);
Method truncateMethod = fs.getClass().getDeclaredMethod("truncate",
new Class[]{Path.class, long.class});
truncateMethod.invoke(fs, new Object[]{pt, newSize});
} catch (NoSuchMethodException e) {
LOGGER.error("the version of hadoop is below 2.7, there is no 'truncate'"
+ " method in FileSystem, It needs to use 'CarbonFile.truncate'.");
CarbonFile carbonFile = FileFactory.getCarbonFile(path, fileType);
carbonFile.truncate(path, newSize);
} catch (Exception e) {
LOGGER.error("Other exception occurred while truncating the file " + e.getMessage());
}
return;
default:
fileChannel = new FileOutputStream(path, true).getChannel();
try {
fileChannel.truncate(newSize);
} finally {
if (fileChannel != null) {
fileChannel.close();
}
}
return;
}
}
/**
* for creating a new Lock file and if it is successfully created
* then in case of abrupt shutdown then the stream to that file will be closed.
*
* @param filePath
* @param fileType
* @return
* @throws IOException
*/
public static boolean createNewLockFile(String filePath, FileType fileType) throws IOException {
return getCarbonFile(filePath).createNewLockFile(filePath, fileType);
}
public enum FileType {
LOCAL, HDFS, ALLUXIO, VIEWFS, S3
}
/**
* below method will be used to update the file path
* for local type
* it removes the file:/ from the path
*
* @param filePath
* @param fileType
* @return updated file path without url for local
*/
public static String getUpdatedFilePath(String filePath, FileType fileType) {
switch (fileType) {
case HDFS:
case ALLUXIO:
case VIEWFS:
case S3:
return filePath;
case LOCAL:
default:
if (filePath != null && !filePath.isEmpty()) {
// If the store path is relative then convert to absolute path.
if (filePath.startsWith("./")) {
try {
return new File(filePath).getCanonicalPath();
} catch (IOException e) {
throw new RuntimeException(e);
}
} else {
Path pathWithoutSchemeAndAuthority =
Path.getPathWithoutSchemeAndAuthority(new Path(filePath));
return pathWithoutSchemeAndAuthority.toString();
}
} else {
return filePath;
}
}
}
/**
* below method will be used to update the file path
* for local type
* it removes the file:/ from the path
*
* @param filePath
* @return updated file path without url for local
*/
public static String getUpdatedFilePath(String filePath) {
FileType fileType = getFileType(filePath);
return getUpdatedFilePath(filePath, fileType);
}
/**
* It computes size of directory
*
* @param filePath
* @return size in bytes
* @throws IOException
*/
public static long getDirectorySize(String filePath) throws IOException {
FileType fileType = getFileType(filePath);
switch (fileType) {
case HDFS:
case ALLUXIO:
case VIEWFS:
case S3:
Path path = new Path(filePath);
FileSystem fs = path.getFileSystem(configuration);
return fs.getContentSummary(path).getLength();
case LOCAL:
default:
filePath = getUpdatedFilePath(filePath, fileType);
File file = new File(filePath);
return FileUtils.sizeOfDirectory(file);
}
}
/**
* This method will create the path object for a given file
*
* @param filePath
* @return
*/
public static Path getPath(String filePath) {
return new Path(filePath);
}
/**
* This method will return the filesystem instance
*
* @param path
* @return
* @throws IOException
*/
public static FileSystem getFileSystem(Path path) throws IOException {
return path.getFileSystem(configuration);
}
public static void createDirectoryAndSetPermission(String directoryPath, FsPermission permission)
throws IOException {
FileFactory.FileType fileType = FileFactory.getFileType(directoryPath);
switch (fileType) {
case S3:
case HDFS:
case VIEWFS:
try {
Path path = new Path(directoryPath);
FileSystem fs = path.getFileSystem(FileFactory.configuration);
if (!fs.exists(path)) {
fs.mkdirs(path);
fs.setPermission(path, permission);
}
} catch (IOException e) {
LOGGER.error("Exception occurred : " + e.getMessage());
throw e;
}
return;
case LOCAL:
default:
directoryPath = FileFactory.getUpdatedFilePath(directoryPath, fileType);
File file = new File(directoryPath);
if (!file.mkdirs()) {
LOGGER.error(" Failed to create directory path " + directoryPath);
}
}
}
/**
* set the file replication
*
* @param path file path
* @param fileType file type
* @param replication replication
* @return true, if success; false, if failed
* @throws IOException if error occurs
*/
public static boolean setReplication(String path, FileFactory.FileType fileType,
short replication) throws IOException {
return getCarbonFile(path, fileType).setReplication(path, replication);
}
/**
* get the default replication
*
* @param path file path
* @param fileType file type
* @return replication
* @throws IOException if error occurs
*/
public static short getDefaultReplication(String path, FileFactory.FileType fileType)
throws IOException {
return getCarbonFile(path, fileType).getDefaultReplication(path);
}
}