You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/07/07 18:08:34 UTC

[GitHub] [hbase] catalin-luca commented on a change in pull request #2002: HBASE-24541 Add support to run LoadIncrementalHFiles in a distributed manner

catalin-luca commented on a change in pull request #2002:
URL: https://github.com/apache/hbase/pull/2002#discussion_r451051217



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFilesJob.java
##########
@@ -0,0 +1,348 @@
+/**
+ * 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.hadoop.hbase.mapreduce;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.HFileLink;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.*;
+
+public class LoadIncrementalHFilesJob extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(LoadIncrementalHFilesJob.class);
+
+  public static final String ROOT_DIR = "loadincrementalhfilesjob.root.dir";
+  public static final String TABLE_NAME = "loadincrementalhfilesjob.table.name";
+  public static final String MAX_MAP_TASK = "loadincrementalhfilesjob.max.map.tasks";
+  public static final String DEPTH = "loadincrementalhfilesjob.depth";
+
+  @Override public int run(String[] args) throws Exception {
+    Configuration configuration = HBaseConfiguration.create(getConf());
+
+    return run(args[0], args[1], configuration);
+  }
+
+  @VisibleForTesting public int run(String rootPath, String table, Configuration configuration)
+      throws Exception {
+    Path basePath = new Path(rootPath);
+    configuration.set(TABLE_NAME, table);
+    configuration.set(ROOT_DIR, basePath.toString());
+
+    if (!bulkload(basePath, table, configuration)) {
+      return 1;
+    }
+
+    return 0;
+  }
+
+  private boolean bulkload(Path rootDir, String table, Configuration configuration)
+      throws Exception {
+    Configuration jobConfiguration = new Configuration(configuration);
+
+    applyNonOverridableOptions(jobConfiguration);
+
+    Job job = Job.getInstance(jobConfiguration, "Bulkload-" + table + "-" + rootDir.toString());
+
+    TableMapReduceUtil.addDependencyJars(job);
+
+    job.setJarByClass(LoadIncrementalHFilesJob.class);
+
+    job.setInputFormatClass(BulkLoadInputFormat.class);
+    job.setOutputFormatClass(NullOutputFormat.class);
+
+    job.setMapperClass(BulkoadMapper.class);
+    job.setMapOutputKeyClass(NullWritable.class);
+    job.setMapOutputValueClass(NullWritable.class);
+
+    job.setNumReduceTasks(0);
+
+    return job.waitForCompletion(true);
+  }
+
+  private static void applyNonOverridableOptions(Configuration configuration) {

Review comment:
       These settings basically account for the fact that the map task is not using a hadoop `OutputFormat` to write its output: 
    - Task timeout is disabled because it is based on the map/reduce idea of progress (which means either writing to the hadoop `Mapper.Context` or the `OutputFormat`). The `BulkloadMapper` code does neither, so it would inevitably timeout.
    - Speculative execution and task retry are disabled because they would fail anyway. A speculated task would attempt to bulkload the same set of files as the original resulting in a failure. A retried task would potentially attempt to bulkload files that were already bulk loaded by the original task.
   
   I don't think an `OutputFormat` can be implemented because there is no way to implement the `OutputCommitter` contract reliably without changes to the region server side. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org