Flink 修改BucketingSink解决小文件的问题

Flink 修改BucketingSink解决小文件的问题

0、背景

     用flink往hdfs写文件的时候,我们一般会用到HDFS Connector 。其实flink里提供了两种HDFS Connector,一个是Bucketing File Sink,另一个是Streaming File Sink。因为我们使用的Hadoop是2.6.5的,对StreamFileSink的支持不太好,所以在我们构建我们的基于Flink的数据传输平台的时候,就选了Bucketing File Sink。
    因为我们有的业务量不大,数据断断续续的从kafka过来,使用BucketingSink的时候就发现HDFS上生成了很多的小文件,这给我们本就不堪重负的namenode和hdfs增加了不少负担,于是我们不得不着手解决问题,当然了,写一个定时任务每天去合并小文件也是一个非常不错的办法。

1、BucketingSink 解析

    要改BucketingSink 之前,得先看看它的逻辑,主要看看invoke这个方法的逻辑吧,这个方法从getBucketPath这里获取要写往的hdfs目录,我们可以重写这个方法来实现基于业务的servicePath和eventTime时间来确定往哪个目录写。通过shouldRoll这个方法,来判断是否该新建一个文件,至于为什么flink要新建这么多文件,其实是为了确保端到端的恰好一次而做的两阶段提交导致了,在体量大的公司根本不会有这样的问题,也难怪社区不关注这一点了。

	public void invoke(T value) throws Exception {
		Path bucketPath = bucketer.getBucketPath(clock, new Path(basePath), value);

		long currentProcessingTime = processingTimeService.getCurrentProcessingTime();

		BucketState<T> bucketState = state.getBucketState(bucketPath);
		if (bucketState == null) {
			bucketState = new BucketState<>(currentProcessingTime);
			state.addBucketState(bucketPath, bucketState);
		}

		if (shouldRoll(bucketState, currentProcessingTime)) {
			openNewPartFile(bucketPath, bucketState);
		}

		bucketState.writer.write(value);
		bucketState.lastWrittenToTime = currentProcessingTime;
	}

    默认的shouldRoll逻辑主要是两个,一个是文件大小达到指定的配置(batchSize)时,一个是长时间没有写入的情况。我们主要是触发了第二个阈值(batchRolloverInterval),所以导致了很多小文件。

	private boolean shouldRoll(BucketState<T> bucketState, long currentProcessingTime) throws IOException {
		boolean shouldRoll = false;
		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
		if (!bucketState.isWriterOpen) {
			shouldRoll = true;
			LOG.debug("BucketingSink {} starting new bucket.", subtaskIndex);
		} else {
			long writePosition = bucketState.writer.getPos();
			if (writePosition > batchSize) {
				shouldRoll = true;
				LOG.debug(
					"BucketingSink {} starting new bucket because file position {} is above batch size {}.",
					subtaskIndex,
					writePosition,
					batchSize);
			} else {
				if (currentProcessingTime - bucketState.creationTime > batchRolloverInterval) {
					shouldRoll = true;
					LOG.debug(
						"BucketingSink {} starting new bucket because file is older than roll over interval {}.",
						subtaskIndex,
						batchRolloverInterval);
				}
			}
		}
		return shouldRoll;
	}

    确定了是否需要滚动文件之后,由一个openNewPartFile的方法来打开文件,可以看到这里就是遍历目录下的所有文件,part-0-0,part-0-1…直到不存在的一个part-0-x为止,然后打开这个part-0-x,开始继续写文件。

private void openNewPartFile(Path bucketPath, BucketState<T> bucketState) throws Exception {
		......
		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
		Path partPath = assemblePartPath(bucketPath, subtaskIndex, bucketState.partCounter);
		while (fs.exists(partPath) ||
				fs.exists(getPendingPathFor(partPath)) ||
				fs.exists(getInProgressPathFor(partPath))) {
			bucketState.partCounter++;
			partPath = assemblePartPath(bucketPath, subtaskIndex, bucketState.partCounter);
		}

		// Record the creation time of the bucket
		bucketState.creationTime = processingTimeService.getCurrentProcessingTime();

		// increase, so we don't have to check for this name next time
		bucketState.partCounter++;
		......
		bucketState.writer.open(fs, inProgressPath);
	}

2、修改

    既然openNewPartFile是无脑打开新文件,我们再它里面加一个判断,让他在遍历的时候,再检查一下文件大小有没有达到我们设定的batchsize,如果没有达到,重新打开那个文件就好了。

private void openNewPartFile(Path bucketPath, BucketState<T> bucketState) throws Exception {
	......
		while (fs.exists(partPath) ||
				fs.exists(getPendingPathFor(partPath)) ||
				fs.exists(getInProgressPathFor(partPath))) {
			bucketState.partCounter++;
			partPath = assemblePartPath(bucketPath, subtaskIndex, bucketState.partCounter);
		}
		Path inProgressPath;
		Path lastPartPath = null;
		Path lastFilePath = null;
		boolean shouldOpenNewFile = true;
		if(bucketState.partCounter>0) {
			lastPartPath = assemblePartPath(bucketPath, subtaskIndex, bucketState.partCounter - 1);
			if(fs.exists(lastPartPath)) {
				lastFilePath = lastPartPath;
			} else if (fs.exists(getPendingPathFor(lastPartPath))){
				lastFilePath=getPendingPathFor(lastPartPath);
			}else if (fs.exists(getInProgressPathFor(lastPartPath))){
				lastFilePath=getInProgressPathFor(lastPartPath);
			}
            shouldOpenNewFile = fs.getFileStatus(lastFilePath).getLen()>=batchSize;           
		}

		if(shouldOpenNewFile){
			bucketState.creationTime = processingTimeService.getCurrentProcessingTime();
			// increase, so we don't have to check for this name next time
			bucketState.partCounter++;
			LOG.debug("Next part path is {}", partPath.toString());
			bucketState.currentFile = partPath.toString();
			inProgressPath=getInProgressPathFor(partPath);
		}else {
			fs.rename(lastFilePath,getInProgressPathFor(lastPartPath));
			inProgressPath = getInProgressPathFor(lastPartPath);
			bucketState.currentFile = lastPartPath.toString();
		}
		......
		bucketState.writer.open(fs, inProgressPath);
		bucketState.isWriterOpen = true;
	}

    改完了这里我们也破坏了flink自带的checkpoint两阶段提交的恰好一次特性,这对于异常恢复的时候,业务方可能会有数据重复,这也是他们不能接受的,好在有一点是我们的checkpoint里会记录正在写的那个文件的有效长度,我们在恢复的时候,就可以读到这个位置,然后把这个位置后面的数据丢掉,也可以变相实现恰好一次。BucketingSink的异常恢复调用关系是这样的initializeState->handleRestoredBucketState->handlePendingInProgressFile。这个handlePendingInProgressFile方法的作用就是在checkpoint的时候,把in-progress文或者pending文件恢复成有效文件。所以我们的改造就是

private void handlePendingInProgressFile(String file, long validLength) {
	......
				if (refTruncate != null) {
					......
				} else {
					Path validLengthFilePath = getValidLengthPathFor(partPath);
					if (!fs.exists(validLengthFilePath) && fs.exists(partPath)) {
						LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, validLength);
//						try (FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath)) {
//							lengthFileOut.writeUTF(Long.toString(validLength));
//						}
						LOG.info("we don't need valid lenth file,so we remove it valid-length file");
						LOG.info("valid length is "+validLength+" bytes");
						LOG.info("last time writing file size is " + fs.getFileStatus(partPath).getLen());
						FSDataInputStream inputStream = fs.open(partPath);
						FSDataOutputStream outputStream = fs.create(getTranPathFor(partPath));

						byte[] buf = new byte[4096];
						long alreadyWriteSize = 0L;
						for(int bytesRead = inputStream.read(buf); bytesRead >= 0 ; bytesRead = inputStream.read(buf)) {
							if(validLength-alreadyWriteSize>=4096) {
								outputStream.write(buf, 0, bytesRead);
								alreadyWriteSize += bytesRead;
							}else if(validLength-alreadyWriteSize<4096 && validLength-alreadyWriteSize>=0){
								outputStream.write(buf,0,(int)(validLength-alreadyWriteSize));
								alreadyWriteSize += (int)(validLength-alreadyWriteSize);
							} else {
								//do nothing because these byte are larger than valid-length
							}
						}
						inputStream.close();
						outputStream.close();

						LOG.info("remove old file and move transation file to writing file");
						fs.delete(partPath,false);
						fs.rename(getTranPathFor(partPath),partPath);

						LOG.info("remove file which partCounter larger than current bucket");
						try {
							Path currentFileParent = partPath.getParent();
							String partName = partPath.getName();
							int currentSubIndex = Integer.parseInt(partName.split("-")[1]);
							int validCounter = Integer.parseInt(partName.split("-")[2].split("\\.")[0]);

							for (FileStatus fileStatus : fs.listStatus(currentFileParent)) {
								int tmpSubIndex = Integer.parseInt(fileStatus.getPath().getName().split("-")[1]);
								int tmpCounter = Integer.parseInt(fileStatus.getPath().getName().split("-")[2].split("\\.")[0]);
								if (tmpSubIndex == currentSubIndex && tmpCounter > validCounter) {
									LOG.info("removing "+fileStatus.getPath());
									fs.delete(fileStatus.getPath(), false);
								}
							}
						}catch (Exception e){
							System.out.println(e.getMessage());
							System.out.println("delete invalid part file failed,please check it ");
						}
					}
				}

			} catch (IOException e) {
				LOG.error("Error while restoring BucketingSink state.", e);
				throw new RuntimeException("Error while restoring BucketingSink state.", e);
			} catch (InvocationTargetException | IllegalAccessException e) {
				LOG.error("Could not invoke truncate.", e);
				throw new RuntimeException("Could not invoke truncate.", e);
			}
		}
	}
	
	private Path getTranPathFor(Path path) {
		return new Path(path.getParent(), path.getName()+".transaction");
	}

    核心逻辑就是在恢复的时候,先读取有效长度,再把最后一个part打开,按字节读取,读取到有效长度之后,后面的脏数据就不读了,把这些已经读出来的数据,写到一个新文件里,最后删除旧part,把临时part重命名成最后一个part,然后打开文件供后续写入。

  • 0
    点赞
  • 12
    收藏
    觉得还不错? 一键收藏
  • 0
    评论

“相关推荐”对你有帮助么?

  • 非常没帮助
  • 没帮助
  • 一般
  • 有帮助
  • 非常有帮助
提交
评论
添加红包

请填写红包祝福语或标题

红包个数最小为10个

红包金额最低5元

当前余额3.43前往充值 >
需支付:10.00
成就一亿技术人!
领取后你会自动成为博主和红包主的粉丝 规则
hope_wisdom
发出的红包
实付
使用余额支付
点击重新获取
扫码支付
钱包余额 0

抵扣说明:

1.余额是钱包充值的虚拟货币,按照1:1的比例进行支付金额的抵扣。
2.余额无法直接购买下载,可以购买VIP、付费专栏及课程。

余额充值